You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by el...@apache.org on 2016/03/03 18:30:45 UTC

[1/7] calcite git commit: [CALCITE-1086] Avoid sending Signature on Execute for updates

Repository: calcite
Updated Branches:
  refs/heads/master 8c90d8621 -> d27e642c4


[CALCITE-1086] Avoid sending Signature on Execute for updates


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

Branch: refs/heads/master
Commit: 1a9be678313267963b82888a00878914d372d01c
Parents: 8c90d86
Author: Josh Elser <el...@apache.org>
Authored: Wed Mar 2 17:43:25 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Wed Mar 2 19:06:30 2016 -0500

----------------------------------------------------------------------
 .../apache/calcite/avatica/AvaticaConnection.java | 18 ++++++++++++++----
 .../calcite/avatica/AvaticaPreparedStatement.java | 10 ++++++----
 .../apache/calcite/avatica/AvaticaStatement.java  | 10 ++++++----
 .../java/org/apache/calcite/avatica/Meta.java     | 18 ++++++++++++++----
 4 files changed, 40 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/1a9be678/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 4076c4e..2d89f45 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
@@ -115,7 +115,7 @@ public abstract class AvaticaConnection implements Connection {
   }
 
   /** Computes the number of retries
-   * {@link AvaticaStatement#executeInternal(Meta.Signature)}
+   * {@link AvaticaStatement#executeInternal(Meta.Signature, boolean)}
    * should retry before failing. */
   long getNumStatementRetries(Properties props) {
     return Long.valueOf(Objects.requireNonNull(props)
@@ -436,11 +436,13 @@ public abstract class AvaticaConnection implements Connection {
    * @param signature     Prepared query
    * @param firstFrame    First frame of rows, or null if we need to execute
    * @param state         The state used to create the given result
+   * @param isUpdate      Was the caller context via {@link PreparedStatement#executeUpdate()}.
    * @return Result set
    * @throws java.sql.SQLException if a database error occurs
    */
   protected ResultSet executeQueryInternal(AvaticaStatement statement,
-      Meta.Signature signature, Meta.Frame firstFrame, QueryState state) throws SQLException {
+      Meta.Signature signature, Meta.Frame firstFrame, QueryState state, boolean isUpdate)
+      throws SQLException {
     // Close the previous open result set, if there is one.
     Meta.Frame frame = firstFrame;
     Meta.Signature signature2 = signature;
@@ -460,8 +462,15 @@ public abstract class AvaticaConnection implements Connection {
       try {
         if (statement.isWrapperFor(AvaticaPreparedStatement.class)) {
           final AvaticaPreparedStatement pstmt = (AvaticaPreparedStatement) statement;
+          Meta.StatementHandle handle = pstmt.handle;
+          if (isUpdate) {
+            // Make a copy of the StatementHandle, nulling out the Signature.
+            // CALCITE-1086 we don't need to send the Signature to the server
+            // when we're only performing an update. Saves on serialization.
+            handle = new Meta.StatementHandle(handle.connectionId, handle.id, null);
+          }
           final Meta.ExecuteResult executeResult =
-              meta.execute(pstmt.handle, pstmt.getParameterValues(),
+              meta.execute(handle, pstmt.getParameterValues(),
                   statement.getFetchSize());
           final MetaResultSet metaResultSet = executeResult.resultSets.get(0);
           frame = metaResultSet.firstFrame;
@@ -577,8 +586,9 @@ public abstract class AvaticaConnection implements Connection {
     final Meta.StatementHandle h = new Meta.StatementHandle(
         metaResultSet.connectionId, metaResultSet.statementId, null);
     final AvaticaStatement statement = lookupStatement(h);
+    // These are all the metadata operations, no updates
     ResultSet resultSet = executeQueryInternal(statement, metaResultSet.signature.sanitize(),
-        metaResultSet.firstFrame, state);
+        metaResultSet.firstFrame, state, false);
     if (metaResultSet.ownStatement) {
       resultSet.getStatement().closeOnCompletion();
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1a9be678/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java
index 4f1c726..f3a950a 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaPreparedStatement.java
@@ -109,7 +109,7 @@ public abstract class AvaticaPreparedStatement
     this.updateCount = -1;
     final Signature sig = getSignature();
     return getConnection().executeQueryInternal(this, sig, null,
-        new QueryState(sig.sql));
+        new QueryState(sig.sql), false);
   }
 
   public ParameterMetaData getParameterMetaData() throws SQLException {
@@ -121,8 +121,8 @@ public abstract class AvaticaPreparedStatement
   }
 
   public long executeLargeUpdate() throws SQLException {
-    getConnection().executeQueryInternal(this, getSignature(), null,
-        new QueryState(getSignature().sql));
+    getConnection().executeQueryInternal(this, null, null,
+        new QueryState(getSignature().sql), true);
     return updateCount;
   }
 
@@ -203,8 +203,10 @@ public abstract class AvaticaPreparedStatement
 
   public boolean execute() throws SQLException {
     this.updateCount = -1;
+    // We don't know if this is actually an update or a query, so call it a query so we pass the
+    // Signature to the server.
     getConnection().executeQueryInternal(this, getSignature(), null,
-        new QueryState(getSignature().sql));
+        new QueryState(getSignature().sql), false);
     // Result set is null for DML or DDL.
     // Result set is closed if user cancelled the query.
     return openResultSet != null && !openResultSet.isClosed();

http://git-wip-us.apache.org/repos/asf/calcite/blob/1a9be678/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 ca73f43..cfd1d45 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaStatement.java
@@ -456,13 +456,14 @@ public abstract class AvaticaStatement
    * Executes a prepared statement.
    *
    * @param signature Parsed statement
+   * @param isUpdate if the execute is for an update
    *
    * @return as specified by {@link java.sql.Statement#execute(String)}
    * @throws java.sql.SQLException if a database error occurs
    */
-  protected boolean executeInternal(Meta.Signature signature)
+  protected boolean executeInternal(Meta.Signature signature, boolean isUpdate)
       throws SQLException {
-    ResultSet resultSet = executeQueryInternal(signature);
+    ResultSet resultSet = executeQueryInternal(signature, isUpdate);
     // user may have cancelled the query
     if (resultSet.isClosed()) {
       return false;
@@ -474,12 +475,13 @@ public abstract class AvaticaStatement
    * Executes a prepared query, closing any previously open result set.
    *
    * @param signature Parsed query
+   * @param isUpdate If the execute is for an update
    * @return Result set
    * @throws java.sql.SQLException if a database error occurs
    */
-  protected ResultSet executeQueryInternal(Meta.Signature signature)
+  protected ResultSet executeQueryInternal(Meta.Signature signature, boolean isUpdate)
       throws SQLException {
-    return connection.executeQueryInternal(this, signature, null, null);
+    return connection.executeQueryInternal(this, signature, null, null, isUpdate);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/1a9be678/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 4f1d56a..a83f2b3 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
@@ -1129,13 +1129,23 @@ public interface Meta {
     }
 
     public Common.StatementHandle toProto() {
-      return Common.StatementHandle.newBuilder().setConnectionId(connectionId)
-          .setId(id).setSignature(signature.toProto()).build();
+      Common.StatementHandle.Builder builder = Common.StatementHandle.newBuilder()
+          .setConnectionId(connectionId).setId(id);
+      if (null != signature) {
+        builder.setSignature(signature.toProto());
+      }
+      return builder.build();
     }
 
     public static StatementHandle fromProto(Common.StatementHandle protoHandle) {
-      return new StatementHandle(protoHandle.getConnectionId(), protoHandle.getId(),
-          Signature.fromProto(protoHandle.getSignature()));
+      final Descriptor desc = protoHandle.getDescriptorForType();
+      // Signature is optional in the update path for executes.
+      Signature signature = null;
+      if (ProtobufService.hasField(protoHandle, desc,
+          Common.StatementHandle.SIGNATURE_FIELD_NUMBER)) {
+        signature = Signature.fromProto(protoHandle.getSignature());
+      }
+      return new StatementHandle(protoHandle.getConnectionId(), protoHandle.getId(), signature);
     }
 
     @Override public int hashCode() {


[7/7] calcite git commit: [CALCITE-1118] Add a noop-JDBC driver for testing Avatica server

Posted by el...@apache.org.
[CALCITE-1118] Add a noop-JDBC driver for testing Avatica server


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

Branch: refs/heads/master
Commit: d27e642c497d406474fd8b543d74836152e8696c
Parents: aecefef
Author: Josh Elser <el...@apache.org>
Authored: Wed Mar 2 17:43:50 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Wed Mar 2 23:26:08 2016 -0500

----------------------------------------------------------------------
 avatica-noop/pom.xml                            | 110 +++
 .../avatica/noop/AvaticaNoopConnection.java     | 256 ++++++
 .../noop/AvaticaNoopDatabaseMetaData.java       | 770 +++++++++++++++++++
 .../calcite/avatica/noop/AvaticaNoopDriver.java |  74 ++
 .../noop/AvaticaNoopParameterMetaData.java      |  85 ++
 .../noop/AvaticaNoopPreparedStatement.java      | 344 +++++++++
 .../avatica/noop/AvaticaNoopResultSet.java      | 661 ++++++++++++++++
 .../noop/AvaticaNoopResultSetMetaData.java      | 133 ++++
 .../avatica/noop/AvaticaNoopStatement.java      | 191 +++++
 .../calcite/avatica/noop/PackageMarker.java     |  37 +
 .../calcite/avatica/noop/package-info.java      |  24 +
 .../resources/META-INF/services/java.sql.Driver |   1 +
 pom.xml                                         |   6 +
 13 files changed, 2692 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/avatica-noop/pom.xml
----------------------------------------------------------------------
diff --git a/avatica-noop/pom.xml b/avatica-noop/pom.xml
new file mode 100644
index 0000000..3a05cdd
--- /dev/null
+++ b/avatica-noop/pom.xml
@@ -0,0 +1,110 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.calcite</groupId>
+    <artifactId>calcite</artifactId>
+    <version>1.7.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>calcite-avatica-noop</artifactId>
+  <packaging>jar</packaging>
+  <name>Calcite Avatica Noop</name>
+  <description>A Noop JDBC driver.</description>
+
+  <properties>
+    <top.dir>${project.basedir}/..</top.dir>
+  </properties>
+
+  <build>
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <groupId>org.eclipse.m2e</groupId>
+          <artifactId>lifecycle-mapping</artifactId>
+          <version>1.0.0</version>
+          <configuration>
+            <lifecycleMappingMetadata>
+              <pluginExecutions>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-checkstyle-plugin</artifactId>
+                    <versionRange>[2.12.1,)</versionRange>
+                    <goals>
+                      <goal>check</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore />
+                  </action>
+                </pluginExecution>
+              </pluginExecutions>
+            </lifecycleMappingMetadata>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+    <plugins>
+      <!-- Parent module has the same plugin and does the work of
+           generating -sources.jar for each project. But without the
+           plugin declared here, IDEs don't know the sources are
+           available. -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>attach-sources</id>
+            <phase>verify</phase>
+            <goals>
+              <goal>jar-no-fork</goal>
+              <goal>test-jar-no-fork</goal>
+            </goals>
+          </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>
+      <plugin>
+        <artifactId>maven-remote-resources-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>non-root-resources</id>
+            <goals>
+              <goal>process</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopConnection.java
----------------------------------------------------------------------
diff --git a/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopConnection.java b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopConnection.java
new file mode 100644
index 0000000..6870717
--- /dev/null
+++ b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopConnection.java
@@ -0,0 +1,256 @@
+/*
+ * 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.noop;
+
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.CallableStatement;
+import java.sql.Clob;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.NClob;
+import java.sql.PreparedStatement;
+import java.sql.SQLClientInfoException;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Savepoint;
+import java.sql.Statement;
+import java.sql.Struct;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Executor;
+
+/**
+ * A {@link Connection} implementation which does nothing.
+ */
+public class AvaticaNoopConnection implements Connection {
+
+  private static final AvaticaNoopConnection INSTANCE = new AvaticaNoopConnection();
+
+  public static AvaticaNoopConnection getInstance() {
+    return INSTANCE;
+  }
+
+  private AvaticaNoopConnection() {}
+
+  private UnsupportedOperationException unsupported() {
+    return new UnsupportedOperationException("Unsupported");
+  }
+
+  @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Statement createStatement() throws SQLException {
+    return AvaticaNoopStatement.getInstance();
+  }
+
+  @Override public PreparedStatement prepareStatement(String sql) throws SQLException {
+    return AvaticaNoopPreparedStatement.getInstance();
+  }
+
+  @Override public CallableStatement prepareCall(String sql) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String nativeSQL(String sql) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setAutoCommit(boolean autoCommit) throws SQLException {}
+
+  @Override public boolean getAutoCommit() throws SQLException {
+    return false;
+  }
+
+  @Override public void commit() throws SQLException {}
+
+  @Override public void rollback() throws SQLException {}
+
+  @Override public void close() throws SQLException {}
+
+  @Override public boolean isClosed() throws SQLException {
+    return true;
+  }
+
+  @Override public DatabaseMetaData getMetaData() throws SQLException {
+    return AvaticaNoopDatabaseMetaData.getInstance();
+  }
+
+  @Override public void setReadOnly(boolean readOnly) throws SQLException {}
+
+  @Override public boolean isReadOnly() throws SQLException {
+    return false;
+  }
+
+  @Override public void setCatalog(String catalog) throws SQLException {}
+
+  @Override public String getCatalog() throws SQLException {
+    return null;
+  }
+
+  @Override public void setTransactionIsolation(int level) throws SQLException {}
+
+  @Override public int getTransactionIsolation() throws SQLException {
+    return 0;
+  }
+
+  @Override public SQLWarning getWarnings() throws SQLException {
+    return null;
+  }
+
+  @Override public void clearWarnings() throws SQLException {}
+
+  @Override public Statement createStatement(int resultSetType, int resultSetConcurrency)
+      throws SQLException {
+    return AvaticaNoopStatement.getInstance();
+  }
+
+  @Override public PreparedStatement prepareStatement(String sql, int resultSetType,
+      int resultSetConcurrency) throws SQLException {
+    return AvaticaNoopPreparedStatement.getInstance();
+  }
+
+  @Override public CallableStatement prepareCall(String sql, int resultSetType,
+      int resultSetConcurrency) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Map<String, Class<?>> getTypeMap() throws SQLException {
+    return Collections.emptyMap();
+  }
+
+  @Override public void setTypeMap(Map<String, Class<?>> map) throws SQLException {}
+
+  @Override public void setHoldability(int holdability) throws SQLException {}
+
+  @Override public int getHoldability() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Savepoint setSavepoint() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Savepoint setSavepoint(String name) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void rollback(Savepoint savepoint) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void releaseSavepoint(Savepoint savepoint) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Statement createStatement(int resultSetType, int resultSetConcurrency,
+      int resultSetHoldability) throws SQLException {
+    return AvaticaNoopStatement.getInstance();
+  }
+
+  @Override public PreparedStatement prepareStatement(String sql, int resultSetType,
+      int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+    return AvaticaNoopPreparedStatement.getInstance();
+  }
+
+  @Override public CallableStatement prepareCall(String sql, int resultSetType,
+      int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys)
+      throws SQLException {
+    return AvaticaNoopPreparedStatement.getInstance();
+  }
+
+  @Override public PreparedStatement prepareStatement(String sql, int[] columnIndexes)
+      throws SQLException {
+    return AvaticaNoopPreparedStatement.getInstance();
+  }
+
+  @Override public PreparedStatement prepareStatement(String sql, String[] columnNames)
+      throws SQLException {
+    return AvaticaNoopPreparedStatement.getInstance();
+  }
+
+  @Override public Clob createClob() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Blob createBlob() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public NClob createNClob() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public SQLXML createSQLXML() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isValid(int timeout) throws SQLException {
+    return true;
+  }
+
+  @Override public void setClientInfo(String name, String value) throws SQLClientInfoException {}
+
+  @Override public void setClientInfo(Properties properties) throws SQLClientInfoException {}
+
+  @Override public String getClientInfo(String name) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Properties getClientInfo() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Array createArrayOf(String typeName, Object[] elements) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Struct createStruct(String typeName, Object[] attributes) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setSchema(String schema) throws SQLException {}
+
+  @Override public String getSchema() throws SQLException {
+    return null;
+  }
+
+  @Override public void abort(Executor executor) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setNetworkTimeout(Executor executor, int milliseconds)
+      throws SQLException {}
+
+  @Override public int getNetworkTimeout() throws SQLException {
+    throw unsupported();
+  }
+
+}
+
+// End AvaticaNoopConnection.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopDatabaseMetaData.java b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopDatabaseMetaData.java
new file mode 100644
index 0000000..2b58590
--- /dev/null
+++ b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopDatabaseMetaData.java
@@ -0,0 +1,770 @@
+/*
+ * 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.noop;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.RowIdLifetime;
+import java.sql.SQLException;
+
+/**
+ * A {@link DatabaseMetaData} implementation which does nothing.
+ */
+public class AvaticaNoopDatabaseMetaData implements DatabaseMetaData {
+
+  private static final AvaticaNoopDatabaseMetaData INSTANCE = new AvaticaNoopDatabaseMetaData();
+
+  public static AvaticaNoopDatabaseMetaData getInstance() {
+    return INSTANCE;
+  }
+
+  private AvaticaNoopDatabaseMetaData() {}
+
+  private UnsupportedOperationException unsupported() {
+    return new UnsupportedOperationException("Unsupported");
+  }
+
+  @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean allProceduresAreCallable() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean allTablesAreSelectable() throws SQLException {
+    return false;
+  }
+
+  @Override public String getURL() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getUserName() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isReadOnly() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean nullsAreSortedHigh() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean nullsAreSortedLow() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean nullsAreSortedAtStart() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean nullsAreSortedAtEnd() throws SQLException {
+    return false;
+  }
+
+  @Override public String getDatabaseProductName() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getDatabaseProductVersion() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getDriverName() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getDriverVersion() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getDriverMajorVersion() {
+    return 0;
+  }
+
+  @Override public int getDriverMinorVersion() {
+    return 0;
+  }
+
+  @Override public boolean usesLocalFiles() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean usesLocalFilePerTable() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsMixedCaseIdentifiers() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean storesUpperCaseIdentifiers() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean storesLowerCaseIdentifiers() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean storesMixedCaseIdentifiers() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsMixedCaseQuotedIdentifiers() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean storesUpperCaseQuotedIdentifiers() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean storesLowerCaseQuotedIdentifiers() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean storesMixedCaseQuotedIdentifiers() throws SQLException {
+    return false;
+  }
+
+  @Override public String getIdentifierQuoteString() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getSQLKeywords() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getNumericFunctions() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getStringFunctions() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getSystemFunctions() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getTimeDateFunctions() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getSearchStringEscape() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getExtraNameCharacters() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean supportsAlterTableWithAddColumn() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsAlterTableWithDropColumn() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsColumnAliasing() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean nullPlusNonNullIsNull() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsConvert() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsConvert(int fromType, int toType) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsTableCorrelationNames() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsDifferentTableCorrelationNames() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsExpressionsInOrderBy() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsOrderByUnrelated() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsGroupBy() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsGroupByUnrelated() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsGroupByBeyondSelect() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsLikeEscapeClause() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsMultipleResultSets() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsMultipleTransactions() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsNonNullableColumns() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsMinimumSQLGrammar() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsCoreSQLGrammar() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsExtendedSQLGrammar() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsANSI92EntryLevelSQL() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsANSI92IntermediateSQL() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsANSI92FullSQL() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsIntegrityEnhancementFacility() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsOuterJoins() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsFullOuterJoins() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsLimitedOuterJoins() throws SQLException {
+    return false;
+  }
+
+  @Override public String getSchemaTerm() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getProcedureTerm() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getCatalogTerm() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isCatalogAtStart() throws SQLException {
+    return false;
+  }
+
+  @Override public String getCatalogSeparator() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean supportsSchemasInDataManipulation() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsSchemasInProcedureCalls() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsSchemasInTableDefinitions() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsSchemasInIndexDefinitions() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsSchemasInPrivilegeDefinitions() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsCatalogsInDataManipulation() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsCatalogsInProcedureCalls() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsCatalogsInTableDefinitions() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsCatalogsInIndexDefinitions() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsCatalogsInPrivilegeDefinitions() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsPositionedDelete() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsPositionedUpdate() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsSelectForUpdate() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsStoredProcedures() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsSubqueriesInComparisons() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsSubqueriesInExists() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsSubqueriesInIns() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsSubqueriesInQuantifieds() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsCorrelatedSubqueries() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsUnion() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsUnionAll() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsOpenCursorsAcrossCommit() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsOpenCursorsAcrossRollback() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsOpenStatementsAcrossCommit() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsOpenStatementsAcrossRollback() throws SQLException {
+    return false;
+  }
+
+  @Override public int getMaxBinaryLiteralLength() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxCharLiteralLength() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxColumnNameLength() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxColumnsInGroupBy() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxColumnsInIndex() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxColumnsInOrderBy() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxColumnsInSelect() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxColumnsInTable() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxConnections() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxCursorNameLength() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxIndexLength() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxSchemaNameLength() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxProcedureNameLength() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxCatalogNameLength() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxRowSize() throws SQLException {
+    return 0;
+  }
+
+  @Override public boolean doesMaxRowSizeIncludeBlobs() throws SQLException {
+    return false;
+  }
+
+  @Override public int getMaxStatementLength() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxStatements() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxTableNameLength() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxTablesInSelect() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getMaxUserNameLength() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getDefaultTransactionIsolation() throws SQLException {
+    return 0;
+  }
+
+  @Override public boolean supportsTransactions() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsTransactionIsolationLevel(int level) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsDataDefinitionAndDataManipulationTransactions()
+      throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsDataManipulationTransactionsOnly() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean dataDefinitionCausesTransactionCommit() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean dataDefinitionIgnoredInTransactions() throws SQLException {
+    return false;
+  }
+
+  @Override public ResultSet getProcedures(String catalog, String schemaPattern,
+      String procedureNamePattern) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getProcedureColumns(String catalog, String schemaPattern,
+      String procedureNamePattern, String columnNamePattern) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getTables(String catalog, String schemaPattern,
+      String tableNamePattern, String[] types) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getSchemas() throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getCatalogs() throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getTableTypes() throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getColumns(String catalog, String schemaPattern,
+      String tableNamePattern, String columnNamePattern) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getColumnPrivileges(String catalog, String schema, String table,
+      String columnNamePattern) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getTablePrivileges(String catalog, String schemaPattern,
+      String tableNamePattern) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getBestRowIdentifier(String catalog, String schema, String table,
+      int scope, boolean nullable) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getVersionColumns(String catalog, String schema, String table)
+      throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getPrimaryKeys(String catalog, String schema, String table)
+      throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getImportedKeys(String catalog, String schema, String table)
+      throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getExportedKeys(String catalog, String schema, String table)
+      throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getCrossReference(String parentCatalog, String parentSchema,
+      String parentTable, String foreignCatalog, String foreignSchema,
+      String foreignTable) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getTypeInfo() throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getIndexInfo(String catalog, String schema, String table,
+      boolean unique, boolean approximate) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public boolean supportsResultSetType(int type) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsResultSetConcurrency(int type, int concurrency)
+      throws SQLException {
+    return false;
+  }
+
+  @Override public boolean ownUpdatesAreVisible(int type) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean ownDeletesAreVisible(int type) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean ownInsertsAreVisible(int type) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean othersUpdatesAreVisible(int type) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean othersDeletesAreVisible(int type) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean othersInsertsAreVisible(int type) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean updatesAreDetected(int type) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean deletesAreDetected(int type) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean insertsAreDetected(int type) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsBatchUpdates() throws SQLException {
+    return false;
+  }
+
+  @Override public ResultSet getUDTs(String catalog, String schemaPattern, String typeNamePattern,
+      int[] types) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Connection getConnection() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean supportsSavepoints() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsNamedParameters() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsMultipleOpenResults() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsGetGeneratedKeys() throws SQLException {
+    return false;
+  }
+
+  @Override public ResultSet getSuperTypes(String catalog, String schemaPattern,
+      String typeNamePattern) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public ResultSet getSuperTables(String catalog, String schemaPattern,
+      String tableNamePattern) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public ResultSet getAttributes(String catalog, String schemaPattern,
+      String typeNamePattern, String attributeNamePattern) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean supportsResultSetHoldability(int holdability) throws SQLException {
+    return false;
+  }
+
+  @Override public int getResultSetHoldability() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getDatabaseMajorVersion() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getDatabaseMinorVersion() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getJDBCMajorVersion() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getJDBCMinorVersion() throws SQLException {
+    return 0;
+  }
+
+  @Override public int getSQLStateType() throws SQLException {
+    return 0;
+  }
+
+  @Override public boolean locatorsUpdateCopy() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean supportsStatementPooling() throws SQLException {
+    return false;
+  }
+
+  @Override public RowIdLifetime getRowIdLifetime() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public ResultSet getSchemas(String catalog, String schemaPattern) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean supportsStoredFunctionsUsingCallSyntax() throws SQLException {
+    return false;
+  }
+
+  @Override public boolean autoCommitFailureClosesAllResultSets() throws SQLException {
+    return false;
+  }
+
+  @Override public ResultSet getClientInfoProperties() throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getFunctions(String catalog, String schemaPattern,
+      String functionNamePattern) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getFunctionColumns(String catalog, String schemaPattern,
+      String functionNamePattern, String columnNamePattern) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public ResultSet getPseudoColumns(String catalog, String schemaPattern,
+      String tableNamePattern, String columnNamePattern) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public boolean generatedKeyAlwaysReturned() throws SQLException {
+    return false;
+  }
+}
+
+// End AvaticaNoopDatabaseMetaData.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopDriver.java
----------------------------------------------------------------------
diff --git a/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopDriver.java b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopDriver.java
new file mode 100644
index 0000000..76f5356
--- /dev/null
+++ b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopDriver.java
@@ -0,0 +1,74 @@
+/*
+ * 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.noop;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.Properties;
+import java.util.logging.Logger;
+
+/**
+ * A Noop JDBC Driver.
+ */
+public class AvaticaNoopDriver implements Driver {
+
+  private static final AvaticaNoopDriver INSTANCE = new AvaticaNoopDriver();
+
+  static {
+    try {
+      DriverManager.registerDriver(INSTANCE);
+    } catch (Exception e) {
+      System.err.println("Failed to register driver");
+      e.printStackTrace(System.err);
+    }
+  }
+
+  @Override public Connection connect(String url, Properties info) throws SQLException {
+    return AvaticaNoopConnection.getInstance();
+  }
+
+  @Override public boolean acceptsURL(String url) throws SQLException {
+    return url.startsWith("jdbc:avatica:noop");
+  }
+
+  @Override public DriverPropertyInfo[] getPropertyInfo(String url, Properties info)
+      throws SQLException {
+    return new DriverPropertyInfo[0];
+  }
+
+  @Override public int getMajorVersion() {
+    return 1;
+  }
+
+  @Override public int getMinorVersion() {
+    return 7;
+  }
+
+  @Override public boolean jdbcCompliant() {
+    return true;
+  }
+
+  @Override public Logger getParentLogger() throws SQLFeatureNotSupportedException {
+    return Logger.getLogger("");
+  }
+}
+
+// End AvaticaNoopDriver.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopParameterMetaData.java
----------------------------------------------------------------------
diff --git a/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopParameterMetaData.java b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopParameterMetaData.java
new file mode 100644
index 0000000..f7117e9
--- /dev/null
+++ b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopParameterMetaData.java
@@ -0,0 +1,85 @@
+/*
+ * 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.noop;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+
+/**
+ * An implementation of {@link ParameterMetaData} which does nothing.
+ */
+public class AvaticaNoopParameterMetaData implements ParameterMetaData {
+
+  private static final AvaticaNoopParameterMetaData INSTANCE = new AvaticaNoopParameterMetaData();
+
+  public static AvaticaNoopParameterMetaData getInstance() {
+    return INSTANCE;
+  }
+
+  private AvaticaNoopParameterMetaData() {}
+
+  private UnsupportedOperationException unsupported() {
+    return new UnsupportedOperationException("Unsupported");
+  }
+
+  @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    return false;
+  }
+
+  @Override public int getParameterCount() throws SQLException {
+    return 0;
+  }
+
+  @Override public int isNullable(int param) throws SQLException {
+    return 0;
+  }
+
+  @Override public boolean isSigned(int param) throws SQLException {
+    return false;
+  }
+
+  @Override public int getPrecision(int param) throws SQLException {
+    return 0;
+  }
+
+  @Override public int getScale(int param) throws SQLException {
+    return 0;
+  }
+
+  @Override public int getParameterType(int param) throws SQLException {
+    return 0;
+  }
+
+  @Override public String getParameterTypeName(int param) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getParameterClassName(int param) throws SQLException {
+    return "java.lang.Object";
+  }
+
+  @Override public int getParameterMode(int param) throws SQLException {
+    return 0;
+  }
+
+}
+
+// End AvaticaNoopParameterMetaData.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopPreparedStatement.java
----------------------------------------------------------------------
diff --git a/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopPreparedStatement.java b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopPreparedStatement.java
new file mode 100644
index 0000000..b8ad8cc
--- /dev/null
+++ b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopPreparedStatement.java
@@ -0,0 +1,344 @@
+/*
+ * 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.noop;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.net.URL;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.NClob;
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
+import java.sql.Ref;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.RowId;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Calendar;
+
+/**
+ * A {@link PreparedStatement} instance which does nothing.
+ */
+public class AvaticaNoopPreparedStatement implements PreparedStatement {
+
+  private static final AvaticaNoopPreparedStatement INSTANCE = new AvaticaNoopPreparedStatement();
+
+  public static AvaticaNoopPreparedStatement getInstance() {
+    return INSTANCE;
+  }
+
+  private AvaticaNoopPreparedStatement() {}
+
+  private UnsupportedOperationException unsupported() {
+    return new UnsupportedOperationException("Unsupported");
+  }
+
+  @Override public ResultSet executeQuery(String sql) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public int executeUpdate(String sql) throws SQLException {
+    return 1;
+  }
+
+  @Override public void close() throws SQLException {}
+
+  @Override public int getMaxFieldSize() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setMaxFieldSize(int max) throws SQLException {}
+
+  @Override public int getMaxRows() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setMaxRows(int max) throws SQLException {}
+
+  @Override public void setEscapeProcessing(boolean enable) throws SQLException {}
+
+  @Override public int getQueryTimeout() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setQueryTimeout(int seconds) throws SQLException {}
+
+  @Override public void cancel() throws SQLException {}
+
+  @Override public SQLWarning getWarnings() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void clearWarnings() throws SQLException {}
+
+  @Override public void setCursorName(String name) throws SQLException {}
+
+  @Override public boolean execute(String sql) throws SQLException {
+    return false;
+  }
+
+  @Override public ResultSet getResultSet() throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public int getUpdateCount() throws SQLException {
+    return 1;
+  }
+
+  @Override public boolean getMoreResults() throws SQLException {
+    return false;
+  }
+
+  @Override public void setFetchDirection(int direction) throws SQLException {}
+
+  @Override public int getFetchDirection() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setFetchSize(int rows) throws SQLException {}
+
+  @Override public int getFetchSize() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getResultSetConcurrency() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getResultSetType() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void addBatch(String sql) throws SQLException {}
+
+  @Override public void clearBatch() throws SQLException {}
+
+  @Override public int[] executeBatch() throws SQLException {
+    return new int[0];
+  }
+
+  @Override public Connection getConnection() throws SQLException {
+    return AvaticaNoopConnection.getInstance();
+  }
+
+  @Override public boolean getMoreResults(int current) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public ResultSet getGeneratedKeys() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
+    return 1;
+  }
+
+  @Override public int executeUpdate(String sql, int[] columnIndexes) throws SQLException {
+    return 1;
+  }
+
+  @Override public int executeUpdate(String sql, String[] columnNames) throws SQLException {
+    return 1;
+  }
+
+  @Override public boolean execute(String sql, int autoGeneratedKeys) throws SQLException {
+    return true;
+  }
+
+  @Override public boolean execute(String sql, int[] columnIndexes) throws SQLException {
+    return true;
+  }
+
+  @Override public boolean execute(String sql, String[] columnNames) throws SQLException {
+    return true;
+  }
+
+  @Override public int getResultSetHoldability() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isClosed() throws SQLException {
+    return false;
+  }
+
+  @Override public void setPoolable(boolean poolable) throws SQLException {}
+
+  @Override public boolean isPoolable() throws SQLException {
+    return true;
+  }
+
+  @Override public void closeOnCompletion() throws SQLException {}
+
+  @Override public boolean isCloseOnCompletion() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    return false;
+  }
+
+  @Override public ResultSet executeQuery() throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public int executeUpdate() throws SQLException {
+    return 1;
+  }
+
+  @Override public void setNull(int parameterIndex, int sqlType) throws SQLException {}
+
+  @Override public void setBoolean(int parameterIndex, boolean x) throws SQLException {}
+
+  @Override public void setByte(int parameterIndex, byte x) throws SQLException {}
+
+  @Override public void setShort(int parameterIndex, short x) throws SQLException {}
+
+  @Override public void setInt(int parameterIndex, int x) throws SQLException {}
+
+  @Override public void setLong(int parameterIndex, long x) throws SQLException {}
+
+  @Override public void setFloat(int parameterIndex, float x) throws SQLException {}
+
+  @Override public void setDouble(int parameterIndex, double x) throws SQLException {}
+
+  @Override public void setBigDecimal(int parameterIndex, BigDecimal x) throws SQLException {}
+
+  @Override public void setString(int parameterIndex, String x) throws SQLException {}
+
+  @Override public void setBytes(int parameterIndex, byte[] x) throws SQLException {}
+
+  @Override public void setDate(int parameterIndex, Date x) throws SQLException {}
+
+  @Override public void setTime(int parameterIndex, Time x) throws SQLException {}
+
+  @Override public void setTimestamp(int parameterIndex, Timestamp x) throws SQLException {}
+
+  @Override public void setAsciiStream(int parameterIndex, InputStream x, int length)
+      throws SQLException {}
+
+  @Override public void setUnicodeStream(int parameterIndex, InputStream x, int length)
+      throws SQLException {}
+
+  @Override public void setBinaryStream(int parameterIndex, InputStream x, int length)
+      throws SQLException {}
+
+  @Override public void clearParameters() throws SQLException {}
+
+  @Override public void setObject(int parameterIndex, Object x, int targetSqlType)
+      throws SQLException {}
+
+  @Override public void setObject(int parameterIndex, Object x) throws SQLException {}
+
+  @Override public boolean execute() throws SQLException {
+    return false;
+  }
+
+  @Override public void addBatch() throws SQLException {}
+
+  @Override public void setCharacterStream(int parameterIndex, Reader reader, int length)
+      throws SQLException {}
+
+  @Override public void setRef(int parameterIndex, Ref x) throws SQLException {}
+
+  @Override public void setBlob(int parameterIndex, Blob x) throws SQLException {}
+
+  @Override public void setClob(int parameterIndex, Clob x) throws SQLException {}
+
+  @Override public void setArray(int parameterIndex, Array x) throws SQLException {}
+
+  @Override public ResultSetMetaData getMetaData() throws SQLException {
+    return AvaticaNoopResultSetMetaData.getInstance();
+  }
+
+  @Override public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException {}
+
+  @Override public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException {}
+
+  @Override public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal)
+      throws SQLException {}
+
+  @Override public void setNull(int parameterIndex, int sqlType, String typeName)
+      throws SQLException {}
+
+  @Override public void setURL(int parameterIndex, URL x) throws SQLException {}
+
+  @Override public ParameterMetaData getParameterMetaData() throws SQLException {
+    return AvaticaNoopParameterMetaData.getInstance();
+  }
+
+  @Override public void setRowId(int parameterIndex, RowId x) throws SQLException {}
+
+  @Override public void setNString(int parameterIndex, String value) throws SQLException {}
+
+  @Override public void setNCharacterStream(int parameterIndex, Reader value, long length)
+      throws SQLException {}
+
+  @Override public void setNClob(int parameterIndex, NClob value) throws SQLException {}
+
+  @Override public void setClob(int parameterIndex, Reader reader, long length)
+      throws SQLException {}
+
+  @Override public void setBlob(int parameterIndex, InputStream inputStream, long length)
+      throws SQLException {}
+
+  @Override public void setNClob(int parameterIndex, Reader reader, long length)
+      throws SQLException {}
+
+  @Override public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException {}
+
+  @Override public void setObject(int parameterIndex, Object x, int targetSqlType,
+      int scaleOrLength)
+      throws SQLException {}
+
+  @Override public void setAsciiStream(int parameterIndex, InputStream x, long length)
+      throws SQLException {}
+
+  @Override public void setBinaryStream(int parameterIndex, InputStream x, long length)
+      throws SQLException {}
+
+  @Override public void setCharacterStream(int parameterIndex, Reader reader, long length)
+      throws SQLException {}
+
+  @Override public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException {}
+
+  @Override public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException {}
+
+  @Override public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException {}
+
+  @Override public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException {}
+
+  @Override public void setClob(int parameterIndex, Reader reader) throws SQLException {}
+
+  @Override public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException {}
+
+  @Override public void setNClob(int parameterIndex, Reader reader) throws SQLException {}
+}
+
+// End AvaticaNoopPreparedStatement.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopResultSet.java
----------------------------------------------------------------------
diff --git a/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopResultSet.java b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopResultSet.java
new file mode 100644
index 0000000..2d71184
--- /dev/null
+++ b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopResultSet.java
@@ -0,0 +1,661 @@
+/*
+ * 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.noop;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.net.URL;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Date;
+import java.sql.NClob;
+import java.sql.Ref;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.RowId;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Calendar;
+import java.util.Map;
+
+/**
+ * A {@link ResultSet} implementation which does nothing.
+ */
+public class AvaticaNoopResultSet implements ResultSet {
+
+  private static final AvaticaNoopResultSet INSTANCE = new AvaticaNoopResultSet();
+
+  public static AvaticaNoopResultSet getInstance() {
+    return INSTANCE;
+  }
+
+  private AvaticaNoopResultSet() {}
+
+  private UnsupportedOperationException unsupported() {
+    return new UnsupportedOperationException("unsupported");
+  }
+
+  @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean next() throws SQLException {
+    return false;
+  }
+
+  @Override public void close() throws SQLException {}
+
+  @Override public boolean wasNull() throws SQLException {
+    return false;
+  }
+
+  @Override public String getString(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean getBoolean(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public byte getByte(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public short getShort(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getInt(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public long getLong(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public float getFloat(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public double getDouble(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public BigDecimal getBigDecimal(int columnIndex, int scale) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public byte[] getBytes(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Date getDate(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Time getTime(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Timestamp getTimestamp(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public InputStream getAsciiStream(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public InputStream getUnicodeStream(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public InputStream getBinaryStream(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getString(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean getBoolean(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public byte getByte(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public short getShort(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getInt(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public long getLong(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public float getFloat(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public double getDouble(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public BigDecimal getBigDecimal(String columnLabel, int scale) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public byte[] getBytes(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Date getDate(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Time getTime(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Timestamp getTimestamp(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public InputStream getAsciiStream(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public InputStream getUnicodeStream(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public InputStream getBinaryStream(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public SQLWarning getWarnings() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void clearWarnings() throws SQLException {}
+
+  @Override public String getCursorName() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public ResultSetMetaData getMetaData() throws SQLException {
+    return AvaticaNoopResultSetMetaData.getInstance();
+  }
+
+  @Override public Object getObject(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Object getObject(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int findColumn(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Reader getCharacterStream(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Reader getCharacterStream(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public BigDecimal getBigDecimal(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public BigDecimal getBigDecimal(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isBeforeFirst() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isAfterLast() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isFirst() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isLast() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void beforeFirst() throws SQLException {}
+
+  @Override public void afterLast() throws SQLException {}
+
+  @Override public boolean first() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean last() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getRow() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean absolute(int row) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean relative(int rows) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean previous() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setFetchDirection(int direction) throws SQLException {}
+
+  @Override public int getFetchDirection() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setFetchSize(int rows) throws SQLException {}
+
+  @Override public int getFetchSize() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getType() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getConcurrency() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean rowUpdated() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean rowInserted() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean rowDeleted() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void updateNull(int columnIndex) throws SQLException {}
+
+  @Override public void updateBoolean(int columnIndex, boolean x) throws SQLException {}
+
+  @Override public void updateByte(int columnIndex, byte x) throws SQLException {}
+
+  @Override public void updateShort(int columnIndex, short x) throws SQLException {}
+
+  @Override public void updateInt(int columnIndex, int x) throws SQLException {}
+
+  @Override public void updateLong(int columnIndex, long x) throws SQLException {}
+
+  @Override public void updateFloat(int columnIndex, float x) throws SQLException {}
+
+  @Override public void updateDouble(int columnIndex, double x) throws SQLException {}
+
+  @Override public void updateBigDecimal(int columnIndex, BigDecimal x) throws SQLException {}
+
+  @Override public void updateString(int columnIndex, String x) throws SQLException {}
+
+  @Override public void updateBytes(int columnIndex, byte[] x) throws SQLException {}
+
+  @Override public void updateDate(int columnIndex, Date x) throws SQLException {}
+
+  @Override public void updateTime(int columnIndex, Time x) throws SQLException {}
+
+  @Override public void updateTimestamp(int columnIndex, Timestamp x) throws SQLException {}
+
+  @Override public void updateAsciiStream(int columnIndex, InputStream x, int length)
+      throws SQLException {}
+
+  @Override public void updateBinaryStream(int columnIndex, InputStream x, int length)
+      throws SQLException {}
+
+  @Override public void updateCharacterStream(int columnIndex, Reader x, int length)
+      throws SQLException {}
+
+  @Override public void updateObject(int columnIndex, Object x, int scaleOrLength)
+      throws SQLException {}
+
+  @Override public void updateObject(int columnIndex, Object x) throws SQLException {}
+
+  @Override public void updateNull(String columnLabel) throws SQLException {}
+
+  @Override public void updateBoolean(String columnLabel, boolean x) throws SQLException {}
+
+  @Override public void updateByte(String columnLabel, byte x) throws SQLException {}
+
+  @Override public void updateShort(String columnLabel, short x) throws SQLException {}
+
+  @Override public void updateInt(String columnLabel, int x) throws SQLException {}
+
+  @Override public void updateLong(String columnLabel, long x) throws SQLException {}
+
+  @Override public void updateFloat(String columnLabel, float x) throws SQLException {}
+
+  @Override public void updateDouble(String columnLabel, double x) throws SQLException {}
+
+  @Override public void updateBigDecimal(String columnLabel, BigDecimal x) throws SQLException {}
+
+  @Override public void updateString(String columnLabel, String x) throws SQLException {}
+
+  @Override public void updateBytes(String columnLabel, byte[] x) throws SQLException {}
+
+  @Override public void updateDate(String columnLabel, Date x) throws SQLException {}
+
+  @Override public void updateTime(String columnLabel, Time x) throws SQLException {}
+
+  @Override public void updateTimestamp(String columnLabel, Timestamp x) throws SQLException {}
+
+  @Override public void updateAsciiStream(String columnLabel, InputStream x, int length)
+      throws SQLException {}
+
+  @Override public void updateBinaryStream(String columnLabel, InputStream x, int length)
+      throws SQLException {}
+
+  @Override public void updateCharacterStream(String columnLabel, Reader reader, int length)
+      throws SQLException {}
+
+  @Override public void updateObject(String columnLabel, Object x, int scaleOrLength)
+      throws SQLException {}
+
+  @Override public void updateObject(String columnLabel, Object x) throws SQLException {}
+
+  @Override public void insertRow() throws SQLException {}
+
+  @Override public void updateRow() throws SQLException {}
+
+  @Override public void deleteRow() throws SQLException {}
+
+  @Override public void refreshRow() throws SQLException {}
+
+  @Override public void cancelRowUpdates() throws SQLException {}
+
+  @Override public void moveToInsertRow() throws SQLException {}
+
+  @Override public void moveToCurrentRow() throws SQLException {}
+
+  @Override public Statement getStatement() throws SQLException {
+    return AvaticaNoopStatement.getInstance();
+  }
+
+  @Override public Object getObject(int columnIndex, Map<String, Class<?>> map)
+      throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Ref getRef(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Blob getBlob(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Clob getClob(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Array getArray(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Object getObject(String columnLabel, Map<String, Class<?>> map)
+      throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Ref getRef(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Blob getBlob(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Clob getClob(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Array getArray(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Date getDate(int columnIndex, Calendar cal) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Date getDate(String columnLabel, Calendar cal) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Time getTime(int columnIndex, Calendar cal) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Time getTime(String columnLabel, Calendar cal) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Timestamp getTimestamp(int columnIndex, Calendar cal) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Timestamp getTimestamp(String columnLabel, Calendar cal) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public URL getURL(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public URL getURL(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void updateRef(int columnIndex, Ref x) throws SQLException {}
+
+  @Override public void updateRef(String columnLabel, Ref x) throws SQLException {}
+
+  @Override public void updateBlob(int columnIndex, Blob x) throws SQLException {}
+
+  @Override public void updateBlob(String columnLabel, Blob x) throws SQLException {}
+
+  @Override public void updateClob(int columnIndex, Clob x) throws SQLException {}
+
+  @Override public void updateClob(String columnLabel, Clob x) throws SQLException {}
+
+  @Override public void updateArray(int columnIndex, Array x) throws SQLException {}
+
+  @Override public void updateArray(String columnLabel, Array x) throws SQLException {}
+
+  @Override public RowId getRowId(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public RowId getRowId(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void updateRowId(int columnIndex, RowId x) throws SQLException {}
+
+  @Override public void updateRowId(String columnLabel, RowId x) throws SQLException {}
+
+  @Override public int getHoldability() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isClosed() throws SQLException {
+    return false;
+  }
+
+  @Override public void updateNString(int columnIndex, String nString) throws SQLException {}
+
+  @Override public void updateNString(String columnLabel, String nString) throws SQLException {}
+
+  @Override public void updateNClob(int columnIndex, NClob nClob) throws SQLException {}
+
+  @Override public void updateNClob(String columnLabel, NClob nClob) throws SQLException {}
+
+  @Override public NClob getNClob(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public NClob getNClob(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public SQLXML getSQLXML(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public SQLXML getSQLXML(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void updateSQLXML(int columnIndex, SQLXML xmlObject) throws SQLException {}
+
+  @Override public void updateSQLXML(String columnLabel, SQLXML xmlObject) throws SQLException {}
+
+  @Override public String getNString(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getNString(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Reader getNCharacterStream(int columnIndex) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public Reader getNCharacterStream(String columnLabel) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void updateNCharacterStream(int columnIndex, Reader x, long length)
+      throws SQLException {}
+
+  @Override public void updateNCharacterStream(String columnLabel, Reader reader, long length)
+      throws SQLException {}
+
+  @Override public void updateAsciiStream(int columnIndex, InputStream x, long length)
+      throws SQLException {}
+
+  @Override public void updateBinaryStream(int columnIndex, InputStream x, long length)
+      throws SQLException {}
+
+  @Override public void updateCharacterStream(int columnIndex, Reader x, long length)
+      throws SQLException {}
+
+  @Override public void updateAsciiStream(String columnLabel, InputStream x, long length)
+      throws SQLException {}
+
+  @Override public void updateBinaryStream(String columnLabel, InputStream x, long length)
+      throws SQLException {}
+
+  @Override public void updateCharacterStream(String columnLabel, Reader reader, long length)
+      throws SQLException {}
+
+  @Override public void updateBlob(int columnIndex, InputStream inputStream, long length)
+      throws SQLException {}
+
+  @Override public void updateBlob(String columnLabel, InputStream inputStream, long length)
+      throws SQLException {}
+
+  @Override public void updateClob(int columnIndex, Reader reader, long length)
+      throws SQLException {}
+
+  @Override public void updateClob(String columnLabel, Reader reader, long length)
+      throws SQLException {}
+
+  @Override public void updateNClob(int columnIndex, Reader reader, long length)
+      throws SQLException {}
+
+  @Override public void updateNClob(String columnLabel, Reader reader, long length)
+      throws SQLException {}
+
+  @Override public void updateNCharacterStream(int columnIndex, Reader x) throws SQLException {}
+
+  @Override public void updateNCharacterStream(String columnLabel, Reader reader)
+      throws SQLException {}
+
+  @Override public void updateAsciiStream(int columnIndex, InputStream x) throws SQLException {}
+
+  @Override public void updateBinaryStream(int columnIndex, InputStream x) throws SQLException {}
+
+  @Override public void updateCharacterStream(int columnIndex, Reader x) throws SQLException {}
+
+  @Override public void updateAsciiStream(String columnLabel, InputStream x) throws SQLException {}
+
+  @Override public void updateBinaryStream(String columnLabel, InputStream x) throws SQLException {}
+
+  @Override public void updateCharacterStream(String columnLabel, Reader reader)
+      throws SQLException {}
+
+  @Override public void updateBlob(int columnIndex, InputStream inputStream) throws SQLException {}
+
+  @Override public void updateBlob(String columnLabel, InputStream inputStream)
+      throws SQLException {}
+
+  @Override public void updateClob(int columnIndex, Reader reader) throws SQLException {}
+
+  @Override public void updateClob(String columnLabel, Reader reader) throws SQLException {}
+
+  @Override public void updateNClob(int columnIndex, Reader reader) throws SQLException {}
+
+  @Override public void updateNClob(String columnLabel, Reader reader) throws SQLException {}
+
+  @Override public <T> T getObject(int columnIndex, Class<T> type) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public <T> T getObject(String columnLabel, Class<T> type) throws SQLException {
+    throw unsupported();
+  }
+
+}
+
+// End AvaticaNoopResultSet.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopResultSetMetaData.java
----------------------------------------------------------------------
diff --git a/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopResultSetMetaData.java b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopResultSetMetaData.java
new file mode 100644
index 0000000..ddc8c3b
--- /dev/null
+++ b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopResultSetMetaData.java
@@ -0,0 +1,133 @@
+/*
+ * 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.noop;
+
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+
+/**
+ * A {@link ResultSetMetaData} implementation which does nothing.
+ */
+public class AvaticaNoopResultSetMetaData implements ResultSetMetaData {
+
+  private static final AvaticaNoopResultSetMetaData INSTANCE = new AvaticaNoopResultSetMetaData();
+
+  public static AvaticaNoopResultSetMetaData getInstance() {
+    return INSTANCE;
+  }
+
+  private AvaticaNoopResultSetMetaData() {}
+
+  private UnsupportedOperationException unsupported() {
+    return new UnsupportedOperationException("Unsupported");
+  }
+
+  @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getColumnCount() throws SQLException {
+    return 0;
+  }
+
+  @Override public boolean isAutoIncrement(int column) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean isCaseSensitive(int column) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean isSearchable(int column) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean isCurrency(int column) throws SQLException {
+    return false;
+  }
+
+  @Override public int isNullable(int column) throws SQLException {
+    return 0;
+  }
+
+  @Override public boolean isSigned(int column) throws SQLException {
+    return false;
+  }
+
+  @Override public int getColumnDisplaySize(int column) throws SQLException {
+    return 0;
+  }
+
+  @Override public String getColumnLabel(int column) throws SQLException {
+    return "";
+  }
+
+  @Override public String getColumnName(int column) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getSchemaName(int column) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getPrecision(int column) throws SQLException {
+    return 0;
+  }
+
+  @Override public int getScale(int column) throws SQLException {
+    return 0;
+  }
+
+  @Override public String getTableName(int column) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public String getCatalogName(int column) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getColumnType(int column) throws SQLException {
+    return 0;
+  }
+
+  @Override public String getColumnTypeName(int column) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isReadOnly(int column) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean isWritable(int column) throws SQLException {
+    return false;
+  }
+
+  @Override public boolean isDefinitelyWritable(int column) throws SQLException {
+    return false;
+  }
+
+  @Override public String getColumnClassName(int column) throws SQLException {
+    throw unsupported();
+  }
+
+}
+
+// End AvaticaNoopResultSetMetaData.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopStatement.java
----------------------------------------------------------------------
diff --git a/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopStatement.java b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopStatement.java
new file mode 100644
index 0000000..079cea4
--- /dev/null
+++ b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/AvaticaNoopStatement.java
@@ -0,0 +1,191 @@
+/*
+ * 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.noop;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.Statement;
+
+/**
+ * A {@link Statement} implementation which does nothing.
+ */
+public class AvaticaNoopStatement implements Statement {
+
+  private static final AvaticaNoopStatement INSTANCE = new AvaticaNoopStatement();
+
+  public static AvaticaNoopStatement getInstance() {
+    return INSTANCE;
+  }
+
+  private AvaticaNoopStatement() {}
+
+  private UnsupportedOperationException unsupported() {
+    return new UnsupportedOperationException("Unsupported");
+  }
+
+  @Override public ResultSet executeQuery(String sql) throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public int executeUpdate(String sql) throws SQLException {
+    return 1;
+  }
+
+  @Override public void close() throws SQLException {}
+
+  @Override public int getMaxFieldSize() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setMaxFieldSize(int max) throws SQLException {}
+
+  @Override public int getMaxRows() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setMaxRows(int max) throws SQLException {}
+
+  @Override public void setEscapeProcessing(boolean enable) throws SQLException {}
+
+  @Override public int getQueryTimeout() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setQueryTimeout(int seconds) throws SQLException {}
+
+  @Override public void cancel() throws SQLException {}
+
+  @Override public SQLWarning getWarnings() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void clearWarnings() throws SQLException {}
+
+  @Override public void setCursorName(String name) throws SQLException {}
+
+  @Override public boolean execute(String sql) throws SQLException {
+    return false;
+  }
+
+  @Override public ResultSet getResultSet() throws SQLException {
+    return AvaticaNoopResultSet.getInstance();
+  }
+
+  @Override public int getUpdateCount() throws SQLException {
+    return 1;
+  }
+
+  @Override public boolean getMoreResults() throws SQLException {
+    return false;
+  }
+
+  @Override public void setFetchDirection(int direction) throws SQLException {}
+
+  @Override public int getFetchDirection() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void setFetchSize(int rows) throws SQLException {}
+
+  @Override public int getFetchSize() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getResultSetConcurrency() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int getResultSetType() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public void addBatch(String sql) throws SQLException {}
+
+  @Override public void clearBatch() throws SQLException {}
+
+  @Override public int[] executeBatch() throws SQLException {
+    return new int[0];
+  }
+
+  @Override public Connection getConnection() throws SQLException {
+    return AvaticaNoopConnection.getInstance();
+  }
+
+  @Override public boolean getMoreResults(int current) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public ResultSet getGeneratedKeys() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
+    return 1;
+  }
+
+  @Override public int executeUpdate(String sql, int[] columnIndexes) throws SQLException {
+    return 1;
+  }
+
+  @Override public int executeUpdate(String sql, String[] columnNames) throws SQLException {
+    return 1;
+  }
+
+  @Override public boolean execute(String sql, int autoGeneratedKeys) throws SQLException {
+    return true;
+  }
+
+  @Override public boolean execute(String sql, int[] columnIndexes) throws SQLException {
+    return true;
+  }
+
+  @Override public boolean execute(String sql, String[] columnNames) throws SQLException {
+    return true;
+  }
+
+  @Override public int getResultSetHoldability() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isClosed() throws SQLException {
+    return false;
+  }
+
+  @Override public void setPoolable(boolean poolable) throws SQLException {}
+
+  @Override public boolean isPoolable() throws SQLException {
+    return true;
+  }
+
+  @Override public void closeOnCompletion() throws SQLException {}
+
+  @Override public boolean isCloseOnCompletion() throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw unsupported();
+  }
+
+  @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw unsupported();
+  }
+}
+
+// End AvaticaNoopStatement.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/PackageMarker.java
----------------------------------------------------------------------
diff --git a/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/PackageMarker.java b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/PackageMarker.java
new file mode 100644
index 0000000..6ae1905
--- /dev/null
+++ b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/PackageMarker.java
@@ -0,0 +1,37 @@
+/*
+ * 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.noop;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * This is a dummy annotation that forces javac to produce output for
+ * otherwise empty package-info.java.
+ *
+ * <p>The result is maven-compiler-plugin can properly identify the scope of
+ * changed files
+ *
+ * <p>See more details in
+ * <a href="https://jira.codehaus.org/browse/MCOMPILER-205">
+ *   maven-compiler-plugin: incremental compilation broken</a>
+ */
+@Retention(RetentionPolicy.SOURCE)
+public @interface PackageMarker {
+}
+
+// End PackageMarker.java


[2/7] calcite git commit: [CALCITE-1092] Use singleton descriptor instances for protobuf field presence checks

Posted by el...@apache.org.
[CALCITE-1092] Use singleton descriptor instances for protobuf field presence checks

Protobuf uses a FieldDescriptor instance to identify an attribute in
some Message and exposes a single hasField method on Message which
accepts a FieldDescriptor. Profiling showed that the Avatica server
was spending a non-zero amount of time in HashMap operations to repeatedly
fetch the appropriate FieldDescriptor each time we were translating
a protobuf to a POJO.

We can reduce this cost by maintaining singletons of these FieldDescriptors
in the POJO class and avoid the repeated HashMap lookups.


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

Branch: refs/heads/master
Commit: 0de38aaa38e0dc43d21c0b2dac4278c67c1dc24c
Parents: ea41b34
Author: Josh Elser <el...@apache.org>
Authored: Wed Mar 2 17:43:37 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Wed Mar 2 19:06:33 2016 -0500

----------------------------------------------------------------------
 .../apache/calcite/avatica/ColumnMetaData.java  |  29 +-
 .../avatica/ConnectionPropertiesImpl.java       |  20 +-
 .../java/org/apache/calcite/avatica/Meta.java   |  45 +--
 .../calcite/avatica/remote/ProtobufService.java |  14 -
 .../apache/calcite/avatica/remote/Service.java  | 335 +++++++++++--------
 .../avatica/remote/ProtobufHandlerTest.java     |   8 +-
 6 files changed, 262 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/0de38aaa/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 60e744d..bcdc228 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
@@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor;
 
 import java.lang.reflect.Type;
 import java.sql.Array;
@@ -47,6 +47,19 @@ import java.util.Objects;
  * <p>(Compare with {@link java.sql.ResultSetMetaData}.)
  */
 public class ColumnMetaData {
+  private static final FieldDescriptor CATALOG_NAME_DESCRIPTOR = Common.ColumnMetaData
+      .getDescriptor().findFieldByNumber(Common.ColumnMetaData.CATALOG_NAME_FIELD_NUMBER);
+  private static final FieldDescriptor SCHEMA_NAME_DESCRIPTOR = Common.ColumnMetaData
+      .getDescriptor().findFieldByNumber(Common.ColumnMetaData.SCHEMA_NAME_FIELD_NUMBER);
+  private static final FieldDescriptor LABEL_DESCRIPTOR = Common.ColumnMetaData
+      .getDescriptor().findFieldByNumber(Common.ColumnMetaData.LABEL_FIELD_NUMBER);
+  private static final FieldDescriptor COLUMN_NAME_DESCRIPTOR = Common.ColumnMetaData
+      .getDescriptor().findFieldByNumber(Common.ColumnMetaData.COLUMN_NAME_FIELD_NUMBER);
+  private static final FieldDescriptor TABLE_NAME_DESCRIPTOR = Common.ColumnMetaData
+      .getDescriptor().findFieldByNumber(Common.ColumnMetaData.TABLE_NAME_FIELD_NUMBER);
+  private static final FieldDescriptor COLUMN_CLASS_NAME_DESCRIPTOR = Common.ColumnMetaData
+      .getDescriptor().findFieldByNumber(Common.ColumnMetaData.COLUMN_CLASS_NAME_FIELD_NUMBER);
+
   public final int ordinal; // 0-based
   public final boolean autoIncrement;
   public final boolean caseSensitive;
@@ -171,36 +184,34 @@ public class ColumnMetaData {
 
   public static ColumnMetaData fromProto(Common.ColumnMetaData proto) {
     AvaticaType nestedType = AvaticaType.fromProto(proto.getType());
-    final Descriptor desc = proto.getDescriptorForType();
 
     String catalogName = null;
-    if (proto.hasField(desc.findFieldByNumber(Common.ColumnMetaData.CATALOG_NAME_FIELD_NUMBER))) {
+    if (proto.hasField(CATALOG_NAME_DESCRIPTOR)) {
       catalogName = proto.getCatalogName();
     }
 
     String schemaName = null;
-    if (proto.hasField(desc.findFieldByNumber(Common.ColumnMetaData.SCHEMA_NAME_FIELD_NUMBER))) {
+    if (proto.hasField(SCHEMA_NAME_DESCRIPTOR)) {
       schemaName = proto.getSchemaName();
     }
 
     String label = null;
-    if (proto.hasField(desc.findFieldByNumber(Common.ColumnMetaData.LABEL_FIELD_NUMBER))) {
+    if (proto.hasField(LABEL_DESCRIPTOR)) {
       label = proto.getLabel();
     }
 
     String columnName = null;
-    if (proto.hasField(desc.findFieldByNumber(Common.ColumnMetaData.COLUMN_NAME_FIELD_NUMBER))) {
+    if (proto.hasField(COLUMN_NAME_DESCRIPTOR)) {
       columnName = proto.getColumnName();
     }
 
     String tableName = null;
-    if (proto.hasField(desc.findFieldByNumber(Common.ColumnMetaData.TABLE_NAME_FIELD_NUMBER))) {
+    if (proto.hasField(TABLE_NAME_DESCRIPTOR)) {
       tableName = proto.getTableName();
     }
 
     String columnClassName = null;
-    if (proto.hasField(
-        desc.findFieldByNumber(Common.ColumnMetaData.COLUMN_CLASS_NAME_FIELD_NUMBER))) {
+    if (proto.hasField(COLUMN_CLASS_NAME_DESCRIPTOR)) {
       columnClassName = proto.getColumnClassName();
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/0de38aaa/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
index e4b01bc..c147ecc 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/ConnectionPropertiesImpl.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/ConnectionPropertiesImpl.java
@@ -17,11 +17,10 @@
 package org.apache.calcite.avatica;
 
 import org.apache.calcite.avatica.proto.Common;
-import org.apache.calcite.avatica.remote.ProtobufService;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor;
 
 import java.sql.Connection;
 import java.sql.SQLException;
@@ -35,6 +34,14 @@ import java.util.Objects;
  * remote.
  */
 public class ConnectionPropertiesImpl implements Meta.ConnectionProperties {
+  private static final FieldDescriptor CATALOG_DESCRIPTOR = Common.ConnectionProperties
+      .getDescriptor().findFieldByNumber(Common.ConnectionProperties.CATALOG_FIELD_NUMBER);
+  private static final FieldDescriptor SCHEMA_DESCRIPTOR = Common.ConnectionProperties
+      .getDescriptor().findFieldByNumber(Common.ConnectionProperties.SCHEMA_FIELD_NUMBER);
+  private static final FieldDescriptor TRANSACTION_ISOLATION_DESCRIPTOR = Common
+      .ConnectionProperties.getDescriptor().findFieldByNumber(
+          Common.ConnectionProperties.TRANSACTION_ISOLATION_FIELD_NUMBER);
+
   private boolean isDirty = false;
   private Boolean autoCommit;
   private Boolean readOnly;
@@ -235,15 +242,13 @@ public class ConnectionPropertiesImpl implements Meta.ConnectionProperties {
   }
 
   public static ConnectionPropertiesImpl fromProto(Common.ConnectionProperties proto) {
-    final Descriptor desc = proto.getDescriptorForType();
-
     String catalog = null;
-    if (ProtobufService.hasField(proto, desc, Common.ConnectionProperties.CATALOG_FIELD_NUMBER)) {
+    if (proto.hasField(CATALOG_DESCRIPTOR)) {
       catalog = proto.getCatalog();
     }
 
     String schema = null;
-    if (ProtobufService.hasField(proto, desc, Common.ConnectionProperties.SCHEMA_FIELD_NUMBER)) {
+    if (proto.hasField(SCHEMA_DESCRIPTOR)) {
       schema = proto.getSchema();
     }
 
@@ -258,8 +263,7 @@ public class ConnectionPropertiesImpl implements Meta.ConnectionProperties {
     }
 
     Integer transactionIsolation = null;
-    if (ProtobufService.hasField(proto, desc,
-        Common.ConnectionProperties.TRANSACTION_ISOLATION_FIELD_NUMBER)) {
+    if (proto.hasField(TRANSACTION_ISOLATION_DESCRIPTOR)) {
       transactionIsolation = Integer.valueOf(proto.getTransactionIsolation());
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/0de38aaa/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 a83f2b3..4cc460c 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/Meta.java
@@ -17,7 +17,6 @@
 package org.apache.calcite.avatica;
 
 import org.apache.calcite.avatica.proto.Common;
-import org.apache.calcite.avatica.remote.ProtobufService;
 import org.apache.calcite.avatica.remote.TypedValue;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
@@ -26,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import com.google.protobuf.ByteString;
-import com.google.protobuf.Descriptors.Descriptor;
 import com.google.protobuf.Descriptors.FieldDescriptor;
 
 import java.lang.reflect.Field;
@@ -476,6 +474,9 @@ public interface Meta {
   /** Information necessary to convert an {@link Iterable} into a
    * {@link org.apache.calcite.avatica.util.Cursor}. */
   final class CursorFactory {
+    private static final FieldDescriptor CLASS_NAME_DESCRIPTOR = Common.CursorFactory.
+        getDescriptor().findFieldByNumber(Common.CursorFactory.CLASS_NAME_FIELD_NUMBER);
+
     public final Style style;
     public final Class clazz;
     @JsonIgnore
@@ -583,10 +584,7 @@ public interface Meta {
       // Reconstruct CursorFactory
       Class<?> clz = null;
 
-      FieldDescriptor clzFieldDesc = proto.getDescriptorForType()
-          .findFieldByNumber(Common.CursorFactory.CLASS_NAME_FIELD_NUMBER);
-
-      if (proto.hasField(clzFieldDesc)) {
+      if (proto.hasField(CLASS_NAME_DESCRIPTOR)) {
         try {
           clz = Class.forName(proto.getClassName());
         } catch (ClassNotFoundException e) {
@@ -632,7 +630,12 @@ public interface Meta {
   }
 
   /** Result of preparing a statement. */
-  class Signature {
+  public class Signature {
+    private static final FieldDescriptor SQL_DESCRIPTOR = Common.Signature
+        .getDescriptor().findFieldByNumber(Common.Signature.SQL_FIELD_NUMBER);
+    private static final FieldDescriptor CURSOR_FACTORY_DESCRIPTOR = Common.Signature
+        .getDescriptor().findFieldByNumber(Common.Signature.CURSOR_FACTORY_FIELD_NUMBER);
+
     public final List<ColumnMetaData> columns;
     public final String sql;
     public final List<AvaticaParameter> parameters;
@@ -731,16 +734,13 @@ public interface Meta {
         parameters.add(AvaticaParameter.fromProto(protoParam));
       }
 
-      final Descriptor desc = protoSignature.getDescriptorForType();
-
       String sql = null;
-      if (ProtobufService.hasField(protoSignature, desc, Common.Signature.SQL_FIELD_NUMBER)) {
+      if (protoSignature.hasField(SQL_DESCRIPTOR)) {
         sql = protoSignature.getSql();
       }
 
       CursorFactory cursorFactory = null;
-      if (ProtobufService.hasField(protoSignature, desc,
-            Common.Signature.CURSOR_FACTORY_FIELD_NUMBER)) {
+      if (protoSignature.hasField(CURSOR_FACTORY_DESCRIPTOR)) {
         cursorFactory = CursorFactory.fromProto(protoSignature.getCursorFactory());
       }
       final Meta.StatementType statementType =
@@ -765,6 +765,10 @@ public interface Meta {
 
   /** A collection of rows. */
   class Frame {
+    private static final FieldDescriptor HAS_ARRAY_VALUE_DESCRIPTOR = Common.ColumnValue
+        .getDescriptor().findFieldByNumber(Common.ColumnValue.HAS_ARRAY_VALUE_FIELD_NUMBER);
+    private static final FieldDescriptor SCALAR_VALUE_DESCRIPTOR = Common.ColumnValue
+        .getDescriptor().findFieldByNumber(Common.ColumnValue.SCALAR_VALUE_FIELD_NUMBER);
     /** Frame that has zero rows and is the last frame. */
     public static final Frame EMPTY =
         new Frame(0, true, Collections.emptyList());
@@ -920,9 +924,8 @@ public interface Meta {
      * @return True if the message is the new style, false otherwise.
      */
     static boolean isNewStyleColumn(Common.ColumnValue column) {
-      final Descriptor desc = column.getDescriptorForType();
-      return ProtobufService.hasField(column, desc, Common.ColumnValue.HAS_ARRAY_VALUE_FIELD_NUMBER)
-          || ProtobufService.hasField(column, desc, Common.ColumnValue.SCALAR_VALUE_FIELD_NUMBER);
+      return column.hasField(HAS_ARRAY_VALUE_DESCRIPTOR)
+          || column.hasField(SCALAR_VALUE_DESCRIPTOR);
     }
 
     /**
@@ -955,8 +958,7 @@ public interface Meta {
       // Verify that we have one or the other (scalar or array)
       validateColumnValue(column);
 
-      if (!ProtobufService.hasField(column, column.getDescriptorForType(),
-          Common.ColumnValue.SCALAR_VALUE_FIELD_NUMBER)) {
+      if (!column.hasField(SCALAR_VALUE_DESCRIPTOR)) {
         // Array
         List<Object> array = new ArrayList<>(column.getArrayValueCount());
         for (Common.TypedValue arrayValue : column.getArrayValueList()) {
@@ -976,8 +978,7 @@ public interface Meta {
      * @throws IllegalArgumentException When the above condition is not met
      */
     static void validateColumnValue(Common.ColumnValue column) {
-      final boolean hasScalar = ProtobufService.hasField(column, column.getDescriptorForType(),
-          Common.ColumnValue.SCALAR_VALUE_FIELD_NUMBER);
+      final boolean hasScalar = column.hasField(SCALAR_VALUE_DESCRIPTOR);
       final boolean hasArrayValue = column.getHasArrayValue();
 
       // These should always be different
@@ -1107,6 +1108,8 @@ public interface Meta {
 
   /** Statement handle. */
   class StatementHandle {
+    private static final FieldDescriptor SIGNATURE_DESCRIPTOR = Common.StatementHandle
+        .getDescriptor().findFieldByNumber(Common.StatementHandle.SIGNATURE_FIELD_NUMBER);
     public final String connectionId;
     public final int id;
 
@@ -1138,11 +1141,9 @@ public interface Meta {
     }
 
     public static StatementHandle fromProto(Common.StatementHandle protoHandle) {
-      final Descriptor desc = protoHandle.getDescriptorForType();
       // Signature is optional in the update path for executes.
       Signature signature = null;
-      if (ProtobufService.hasField(protoHandle, desc,
-          Common.StatementHandle.SIGNATURE_FIELD_NUMBER)) {
+      if (protoHandle.hasField(SIGNATURE_DESCRIPTOR)) {
         signature = Signature.fromProto(protoHandle.getSignature());
       }
       return new StatementHandle(protoHandle.getConnectionId(), protoHandle.getId(), signature);

http://git-wip-us.apache.org/repos/asf/calcite/blob/0de38aaa/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java
index 741c34c..56ba125 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufService.java
@@ -16,8 +16,6 @@
  */
 package org.apache.calcite.avatica.remote;
 
-import com.google.protobuf.Descriptors.Descriptor;
-
 import com.google.protobuf.Message;
 
 /**
@@ -112,18 +110,6 @@ public abstract class ProtobufService extends AbstractService {
   }
 
   /**
-   * Determines whether the given message has the field, denoted by the provided number, set.
-   *
-   * @param msg The protobuf message
-   * @param desc The descriptor for the message
-   * @param fieldNum The identifier for the field
-   * @return True if the message contains the field, false otherwise
-   */
-  public static boolean hasField(Message msg, Descriptor desc, int fieldNum) {
-    return msg.hasField(desc.findFieldByNumber(fieldNum));
-  }
-
-  /**
    * Checks if the provided {@link Message} is an instance of the Class given by
    * <code>expectedType</code>. Throws an IllegalArgumentException if the message is not of the
    * expected type, otherwise, it returns the message cast as the expected type.

http://git-wip-us.apache.org/repos/asf/calcite/blob/0de38aaa/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 fb3b379..aee5b29 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
@@ -32,7 +32,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 
-import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor;
 import com.google.protobuf.Message;
 
 import java.io.PrintWriter;
@@ -172,6 +172,8 @@ public interface Service {
   /** Request for
    * {@link org.apache.calcite.avatica.Meta#getCatalogs(Meta.ConnectionHandle)}. */
   class CatalogsRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.CatalogsRequest.
+        getDescriptor().findFieldByNumber(Requests.CatalogsRequest.CONNECTION_ID_FIELD_NUMBER);
     public final String connectionId;
 
     public CatalogsRequest() {
@@ -190,11 +192,8 @@ public interface Service {
     @Override CatalogsRequest deserialize(Message genericMsg) {
       final Requests.CatalogsRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.CatalogsRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
-
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.CatalogsRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
@@ -227,6 +226,10 @@ public interface Service {
   /** Request for
    * {@link org.apache.calcite.avatica.Meta#getDatabaseProperties(Meta.ConnectionHandle)}. */
   class DatabasePropertyRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR =
+        Requests.DatabasePropertyRequest.getDescriptor().
+        findFieldByNumber(Requests.DatabasePropertyRequest.CONNECTION_ID_FIELD_NUMBER);
+
     public final String connectionId;
 
     public DatabasePropertyRequest() {
@@ -245,11 +248,9 @@ public interface Service {
     @Override DatabasePropertyRequest deserialize(Message genericMsg) {
       final Requests.DatabasePropertyRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.DatabasePropertyRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.DatabasePropertyRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
@@ -283,6 +284,13 @@ public interface Service {
   /** Request for
    * {@link Meta#getSchemas(Meta.ConnectionHandle, String, Meta.Pat)}. */
   class SchemasRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.SchemasRequest.
+        getDescriptor().findFieldByNumber(Requests.SchemasRequest.CONNECTION_ID_FIELD_NUMBER);
+    private static final FieldDescriptor CATALOG_DESCRIPTOR = Requests.SchemasRequest.
+        getDescriptor().findFieldByNumber(Requests.SchemasRequest.CATALOG_FIELD_NUMBER);
+    private static final FieldDescriptor SCHEMA_PATTERN_DESCRIPTOR = Requests.SchemasRequest.
+        getDescriptor().findFieldByNumber(Requests.SchemasRequest.SCHEMA_PATTERN_FIELD_NUMBER);
+
     public final String connectionId;
     public final String catalog;
     public final String schemaPattern;
@@ -309,21 +317,19 @@ public interface Service {
     @Override SchemasRequest deserialize(Message genericMsg) {
       final Requests.SchemasRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.SchemasRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc, Requests.SchemasRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
       String catalog = null;
-      if (ProtobufService.hasField(msg, desc, Requests.SchemasRequest.CATALOG_FIELD_NUMBER)) {
+      if (msg.hasField(CATALOG_DESCRIPTOR)) {
         catalog = msg.getCatalog();
       }
 
       String schemaPattern = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.SchemasRequest.SCHEMA_PATTERN_FIELD_NUMBER)) {
+      if (msg.hasField(SCHEMA_PATTERN_DESCRIPTOR)) {
         schemaPattern = msg.getSchemaPattern();
       }
 
@@ -366,6 +372,15 @@ public interface Service {
    * {@link Meta#getTables(Meta.ConnectionHandle, String, org.apache.calcite.avatica.Meta.Pat, org.apache.calcite.avatica.Meta.Pat, java.util.List)}
    */
   class TablesRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.TablesRequest.
+        getDescriptor().findFieldByNumber(Requests.TablesRequest.CONNECTION_ID_FIELD_NUMBER);
+    private static final FieldDescriptor CATALOG_DESCRIPTOR = Requests.TablesRequest.
+        getDescriptor().findFieldByNumber(Requests.TablesRequest.CATALOG_FIELD_NUMBER);
+    private static final FieldDescriptor SCHEMA_PATTERN_DESCRIPTOR = Requests.TablesRequest.
+        getDescriptor().findFieldByNumber(Requests.TablesRequest.SCHEMA_PATTERN_FIELD_NUMBER);
+    private static final FieldDescriptor TABLE_NAME_PATTERN_DESCRIPTOR = Requests.TablesRequest.
+        getDescriptor().findFieldByNumber(Requests.TablesRequest.TABLE_NAME_PATTERN_FIELD_NUMBER);
+
     public final String connectionId;
     public final String catalog;
     public final String schemaPattern;
@@ -400,26 +415,24 @@ public interface Service {
     @Override Request deserialize(Message genericMsg) {
       final Requests.TablesRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.TablesRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc, Requests.TablesRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
       String catalog = null;
-      if (ProtobufService.hasField(msg, desc, Requests.TablesRequest.CATALOG_FIELD_NUMBER)) {
+      if (msg.hasField(CATALOG_DESCRIPTOR)) {
         catalog = msg.getCatalog();
       }
 
       String schemaPattern = null;
-      if (ProtobufService.hasField(msg, desc, Requests.TablesRequest.SCHEMA_PATTERN_FIELD_NUMBER)) {
+      if (msg.hasField(SCHEMA_PATTERN_DESCRIPTOR)) {
         schemaPattern = msg.getSchemaPattern();
       }
 
       String tableNamePattern = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.TablesRequest.TABLE_NAME_PATTERN_FIELD_NUMBER)) {
+      if (msg.hasField(TABLE_NAME_PATTERN_DESCRIPTOR)) {
         tableNamePattern = msg.getTableNamePattern();
       }
 
@@ -483,6 +496,8 @@ public interface Service {
    * Request for {@link Meta#getTableTypes(Meta.ConnectionHandle)}.
    */
   class TableTypesRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.TableTypesRequest.
+        getDescriptor().findFieldByNumber(Requests.TableTypesRequest.CONNECTION_ID_FIELD_NUMBER);
     public final String connectionId;
 
     public TableTypesRequest() {
@@ -501,11 +516,9 @@ public interface Service {
     @Override TableTypesRequest deserialize(Message genericMsg) {
       final Requests.TableTypesRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.TableTypesRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.TableTypesRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
@@ -538,6 +551,17 @@ public interface Service {
    * {@link Meta#getColumns(Meta.ConnectionHandle, String, org.apache.calcite.avatica.Meta.Pat, org.apache.calcite.avatica.Meta.Pat, org.apache.calcite.avatica.Meta.Pat)}.
    */
   class ColumnsRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.ColumnsRequest.
+        getDescriptor().findFieldByNumber(Requests.ColumnsRequest.CONNECTION_ID_FIELD_NUMBER);
+    private static final FieldDescriptor CATALOG_DESCRIPTOR = Requests.ColumnsRequest.
+        getDescriptor().findFieldByNumber(Requests.ColumnsRequest.CATALOG_FIELD_NUMBER);
+    private static final FieldDescriptor SCHEMA_PATTERN_DESCRIPTOR = Requests.ColumnsRequest.
+        getDescriptor().findFieldByNumber(Requests.ColumnsRequest.SCHEMA_PATTERN_FIELD_NUMBER);
+    private static final FieldDescriptor TABLE_NAME_PATTERN_DESCRIPTOR = Requests.ColumnsRequest.
+        getDescriptor().findFieldByNumber(Requests.ColumnsRequest.TABLE_NAME_PATTERN_FIELD_NUMBER);
+    private static final FieldDescriptor COLUMN_NAME_PATTERN_DESCRIPTOR = Requests.ColumnsRequest.
+        getDescriptor().findFieldByNumber(Requests.ColumnsRequest.COLUMN_NAME_PATTERN_FIELD_NUMBER);
+
     public final String connectionId;
     public final String catalog;
     public final String schemaPattern;
@@ -572,33 +596,29 @@ public interface Service {
     @Override ColumnsRequest deserialize(Message genericMsg) {
       final Requests.ColumnsRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.ColumnsRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc, Requests.ColumnsRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
       String catalog = null;
-      if (ProtobufService.hasField(msg, desc, Requests.ColumnsRequest.CATALOG_FIELD_NUMBER)) {
+      if (msg.hasField(CATALOG_DESCRIPTOR)) {
         catalog = msg.getCatalog();
       }
 
       String schemaPattern = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.ColumnsRequest.SCHEMA_PATTERN_FIELD_NUMBER)) {
+      if (msg.hasField(SCHEMA_PATTERN_DESCRIPTOR)) {
         schemaPattern = msg.getSchemaPattern();
       }
 
       String tableNamePattern = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.ColumnsRequest.TABLE_NAME_PATTERN_FIELD_NUMBER)) {
+      if (msg.hasField(TABLE_NAME_PATTERN_DESCRIPTOR)) {
         tableNamePattern = msg.getTableNamePattern();
       }
 
       String columnNamePattern = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.ColumnsRequest.COLUMN_NAME_PATTERN_FIELD_NUMBER)) {
+      if (msg.hasField(COLUMN_NAME_PATTERN_DESCRIPTOR)) {
         columnNamePattern = msg.getColumnNamePattern();
       }
 
@@ -652,6 +672,8 @@ public interface Service {
   /** Request for
    * {@link Meta#getTypeInfo(Meta.ConnectionHandle)}. */
   class TypeInfoRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.TypeInfoRequest.
+        getDescriptor().findFieldByNumber(Requests.TypeInfoRequest.CONNECTION_ID_FIELD_NUMBER);
     public final String connectionId;
 
     public TypeInfoRequest() {
@@ -670,11 +692,9 @@ public interface Service {
     @Override TypeInfoRequest deserialize(Message genericMsg) {
       final Requests.TypeInfoRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.TypeInfoRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.TypeInfoRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
@@ -716,6 +736,15 @@ public interface Service {
    * {@link Meta#getTableTypes(Meta.ConnectionHandle)}
    * return this response. */
   class ResultSetResponse extends Response {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Responses.ResultSetResponse.
+        getDescriptor().findFieldByNumber(Responses.ResultSetResponse.CONNECTION_ID_FIELD_NUMBER);
+    private static final FieldDescriptor SIGNATURE_DESCRIPTOR = Responses.ResultSetResponse.
+        getDescriptor().findFieldByNumber(Responses.ResultSetResponse.SIGNATURE_FIELD_NUMBER);
+    private static final FieldDescriptor FIRST_FRAME_DESCRIPTOR = Responses.ResultSetResponse.
+        getDescriptor().findFieldByNumber(Responses.ResultSetResponse.FIRST_FRAME_FIELD_NUMBER);
+    private static final FieldDescriptor METADATA_DESCRIPTOR = Responses.ResultSetResponse.
+        getDescriptor().findFieldByNumber(Responses.ResultSetResponse.METADATA_FIELD_NUMBER);
+
     public final String connectionId;
     public final int statementId;
     public final boolean ownStatement;
@@ -760,27 +789,23 @@ public interface Service {
     }
 
     static ResultSetResponse fromProto(Responses.ResultSetResponse msg) {
-      final Descriptor desc = msg.getDescriptorForType();
-
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Responses.ResultSetResponse.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
       Meta.Signature signature = null;
-      if (ProtobufService.hasField(msg, desc, Responses.ResultSetResponse.SIGNATURE_FIELD_NUMBER)) {
+      if (msg.hasField(SIGNATURE_DESCRIPTOR)) {
         signature = Meta.Signature.fromProto(msg.getSignature());
       }
 
       Meta.Frame frame = null;
-      if (ProtobufService.hasField(msg, desc,
-          Responses.ResultSetResponse.FIRST_FRAME_FIELD_NUMBER)) {
+      if (msg.hasField(FIRST_FRAME_DESCRIPTOR)) {
         frame = Meta.Frame.fromProto(msg.getFirstFrame());
       }
 
       RpcMetadataResponse metadata = null;
-      if (ProtobufService.hasField(msg, desc, Responses.ResultSetResponse.METADATA_FIELD_NUMBER)) {
+      if (msg.hasField(METADATA_DESCRIPTOR)) {
         metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
       }
 
@@ -840,6 +865,13 @@ public interface Service {
   /** Request for
    * {@link Meta#prepareAndExecute(Meta.StatementHandle, String, long, Meta.PrepareCallback)}. */
   class PrepareAndExecuteRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.
+        PrepareAndExecuteRequest.getDescriptor().findFieldByNumber(
+            Requests.PrepareAndExecuteRequest.CONNECTION_ID_FIELD_NUMBER);
+    private static final FieldDescriptor SQL_DESCRIPTOR = Requests.
+        PrepareAndExecuteRequest.getDescriptor().findFieldByNumber(
+            Requests.PrepareAndExecuteRequest.SQL_FIELD_NUMBER);
+
     public final String connectionId;
     public final String sql;
     public final long maxRowCount;
@@ -871,16 +903,14 @@ public interface Service {
     @Override PrepareAndExecuteRequest deserialize(Message genericMsg) {
       final Requests.PrepareAndExecuteRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.PrepareAndExecuteRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.PrepareAndExecuteRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
       String sql = null;
-      if (ProtobufService.hasField(msg, desc, Requests.PrepareAndExecuteRequest.SQL_FIELD_NUMBER)) {
+      if (msg.hasField(SQL_DESCRIPTOR)) {
         sql = msg.getSql();
       }
 
@@ -926,6 +956,8 @@ public interface Service {
   /** Request for
    * {@link org.apache.calcite.avatica.Meta#execute}. */
   class ExecuteRequest extends Request {
+    private static final FieldDescriptor STATEMENT_HANDLE_DESCRIPTOR = Requests.ExecuteRequest.
+        getDescriptor().findFieldByNumber(Requests.ExecuteRequest.STATEMENTHANDLE_FIELD_NUMBER);
     public final Meta.StatementHandle statementHandle;
     public final List<TypedValue> parameterValues;
     public final long maxRowCount;
@@ -953,11 +985,9 @@ public interface Service {
     @Override ExecuteRequest deserialize(Message genericMsg) {
       final Requests.ExecuteRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.ExecuteRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       Meta.StatementHandle statemetnHandle = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.ExecuteRequest.STATEMENTHANDLE_FIELD_NUMBER)) {
+      if (msg.hasField(STATEMENT_HANDLE_DESCRIPTOR)) {
         statemetnHandle = Meta.StatementHandle.fromProto(msg.getStatementHandle());
       }
 
@@ -1017,6 +1047,8 @@ public interface Service {
   /** Response to a
    * {@link org.apache.calcite.avatica.remote.Service.PrepareAndExecuteRequest}. */
   class ExecuteResponse extends Response {
+    private static final FieldDescriptor METADATA_DESCRIPTOR = Responses.ExecuteResponse.
+        getDescriptor().findFieldByNumber(Responses.ExecuteResponse.METADATA_FIELD_NUMBER);
     public final List<ResultSetResponse> results;
     public boolean missingStatement = false;
     public final RpcMetadataResponse rpcMetadata;
@@ -1038,7 +1070,6 @@ public interface Service {
     @Override ExecuteResponse deserialize(Message genericMsg) {
       final Responses.ExecuteResponse msg = ProtobufService.castProtobufMessage(genericMsg,
           Responses.ExecuteResponse.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       List<Responses.ResultSetResponse> msgResults = msg.getResultsList();
       List<ResultSetResponse> copiedResults = new ArrayList<>(msgResults.size());
@@ -1048,7 +1079,7 @@ public interface Service {
       }
 
       RpcMetadataResponse metadata = null;
-      if (ProtobufService.hasField(msg, desc, Responses.ExecuteResponse.METADATA_FIELD_NUMBER)) {
+      if (msg.hasField(METADATA_DESCRIPTOR)) {
         metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
       }
 
@@ -1089,6 +1120,10 @@ public interface Service {
   /** Request for
    * {@link Meta#prepare(Meta.ConnectionHandle, String, long)}. */
   class PrepareRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.PrepareRequest.
+        getDescriptor().findFieldByNumber(Requests.PrepareRequest.CONNECTION_ID_FIELD_NUMBER);
+    private static final FieldDescriptor SQL_DESCRIPTOR = Requests.PrepareRequest.
+        getDescriptor().findFieldByNumber(Requests.PrepareRequest.SQL_FIELD_NUMBER);
     public final String connectionId;
     public final String sql;
     public final long maxRowCount;
@@ -1116,15 +1151,14 @@ public interface Service {
     @Override PrepareRequest deserialize(Message genericMsg) {
       final Requests.PrepareRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.PrepareRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc, Requests.PrepareRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
       String sql = null;
-      if (ProtobufService.hasField(msg, desc, Requests.PrepareRequest.SQL_FIELD_NUMBER)) {
+      if (msg.hasField(SQL_DESCRIPTOR)) {
         sql = msg.getSql();
       }
 
@@ -1165,6 +1199,8 @@ public interface Service {
   /** Response from
    * {@link org.apache.calcite.avatica.remote.Service.PrepareRequest}. */
   class PrepareResponse extends Response {
+    private static final FieldDescriptor METADATA_DESCRIPTOR = Responses.PrepareResponse.
+        getDescriptor().findFieldByNumber(Responses.PrepareResponse.METADATA_FIELD_NUMBER);
     public final Meta.StatementHandle statement;
     public final RpcMetadataResponse rpcMetadata;
 
@@ -1184,10 +1220,9 @@ public interface Service {
     @Override PrepareResponse deserialize(Message genericMsg) {
       final Responses.PrepareResponse msg = ProtobufService.castProtobufMessage(genericMsg,
           Responses.PrepareResponse.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       RpcMetadataResponse metadata = null;
-      if (ProtobufService.hasField(msg, desc, Responses.PrepareResponse.METADATA_FIELD_NUMBER)) {
+      if (msg.hasField(METADATA_DESCRIPTOR)) {
         metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
       }
 
@@ -1226,6 +1261,8 @@ public interface Service {
   /** Request for
    * {@link Meta#fetch}. */
   class FetchRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.FetchRequest.
+        getDescriptor().findFieldByNumber(Requests.FetchRequest.CONNECTION_ID_FIELD_NUMBER);
     public final String connectionId;
     public final int statementId;
     public final long offset;
@@ -1259,10 +1296,9 @@ public interface Service {
     @Override FetchRequest deserialize(Message genericMsg) {
       final Requests.FetchRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.FetchRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc, Requests.FetchRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
@@ -1306,6 +1342,8 @@ public interface Service {
   /** Response from
    * {@link org.apache.calcite.avatica.remote.Service.FetchRequest}. */
   class FetchResponse extends Response {
+    private static final FieldDescriptor METADATA_DESCRIPTOR = Responses.FetchResponse.
+        getDescriptor().findFieldByNumber(Responses.FetchResponse.METADATA_FIELD_NUMBER);
     public final Meta.Frame frame;
     public boolean missingStatement = false;
     public boolean missingResults = false;
@@ -1330,10 +1368,9 @@ public interface Service {
     @Override FetchResponse deserialize(Message genericMsg) {
       final Responses.FetchResponse msg = ProtobufService.castProtobufMessage(genericMsg,
           Responses.FetchResponse.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       RpcMetadataResponse metadata = null;
-      if (ProtobufService.hasField(msg, desc, Responses.FetchResponse.METADATA_FIELD_NUMBER)) {
+      if (msg.hasField(METADATA_DESCRIPTOR)) {
         metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
       }
 
@@ -1375,6 +1412,9 @@ public interface Service {
   /** Request for
    * {@link org.apache.calcite.avatica.Meta#createStatement(org.apache.calcite.avatica.Meta.ConnectionHandle)}. */
   class CreateStatementRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.CreateStatementRequest.
+        getDescriptor().findFieldByNumber(
+            Requests.CreateStatementRequest.CONNECTION_ID_FIELD_NUMBER);
     public final String connectionId;
 
     CreateStatementRequest() {
@@ -1394,11 +1434,9 @@ public interface Service {
     @Override CreateStatementRequest deserialize(Message genericMsg) {
       final Requests.CreateStatementRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.CreateStatementRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.CreateStatementRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
@@ -1432,6 +1470,12 @@ public interface Service {
   /** Response from
    * {@link org.apache.calcite.avatica.remote.Service.CreateStatementRequest}. */
   class CreateStatementResponse extends Response {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Responses.
+        CreateStatementResponse.getDescriptor().findFieldByNumber(
+            Responses.CreateStatementResponse.CONNECTION_ID_FIELD_NUMBER);
+    private static final FieldDescriptor METADATA_DESCRIPTOR = Responses.
+        CreateStatementResponse.getDescriptor().findFieldByNumber(
+            Responses.CreateStatementResponse.METADATA_FIELD_NUMBER);
     public final String connectionId;
     public final int statementId;
     public final RpcMetadataResponse rpcMetadata;
@@ -1455,17 +1499,13 @@ public interface Service {
     @Override CreateStatementResponse deserialize(Message genericMsg) {
       final Responses.CreateStatementResponse msg = ProtobufService.castProtobufMessage(genericMsg,
           Responses.CreateStatementResponse.class);
-      final Descriptor desc = msg.getDescriptorForType();
-
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Responses.CreateStatementResponse.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
       RpcMetadataResponse metadata = null;
-      if (ProtobufService.hasField(msg, desc,
-          Responses.CreateStatementResponse.METADATA_FIELD_NUMBER)) {
+      if (msg.hasField(METADATA_DESCRIPTOR)) {
         metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
       }
 
@@ -1509,6 +1549,9 @@ public interface Service {
   /** Request for
    * {@link org.apache.calcite.avatica.Meta#closeStatement(org.apache.calcite.avatica.Meta.StatementHandle)}. */
   class CloseStatementRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.CloseStatementRequest.
+        getDescriptor().findFieldByNumber(
+            Requests.CloseStatementRequest.CONNECTION_ID_FIELD_NUMBER);
     public final String connectionId;
     public final int statementId;
 
@@ -1532,11 +1575,9 @@ public interface Service {
     @Override CloseStatementRequest deserialize(Message genericMsg) {
       final Requests.CloseStatementRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.CloseStatementRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.CloseStatementRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
@@ -1571,6 +1612,10 @@ public interface Service {
   /** Response from
    * {@link org.apache.calcite.avatica.remote.Service.CloseStatementRequest}. */
   class CloseStatementResponse extends Response {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Responses.
+        CloseStatementResponse.getDescriptor().findFieldByNumber(
+            Responses.CloseStatementResponse.METADATA_FIELD_NUMBER);
+
     public final RpcMetadataResponse rpcMetadata;
 
     public CloseStatementResponse() {
@@ -1585,11 +1630,8 @@ public interface Service {
     @Override CloseStatementResponse deserialize(Message genericMsg) {
       final Responses.CloseStatementResponse msg = ProtobufService.castProtobufMessage(genericMsg,
           Responses.CloseStatementResponse.class);
-      final Descriptor desc = msg.getDescriptorForType();
-
       RpcMetadataResponse metadata = null;
-      if (ProtobufService.hasField(msg, desc,
-          Responses.CloseStatementResponse.METADATA_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
       }
 
@@ -1623,6 +1665,9 @@ public interface Service {
   /** Request for
    * {@link Meta#openConnection}. */
   class OpenConnectionRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.OpenConnectionRequest
+        .getDescriptor().findFieldByNumber(
+            Requests.OpenConnectionRequest.CONNECTION_ID_FIELD_NUMBER);
     public final String connectionId;
     public final Map<String, String> info;
 
@@ -1670,11 +1715,9 @@ public interface Service {
     @Override Request deserialize(Message genericMsg) {
       final Requests.OpenConnectionRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.OpenConnectionRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.OpenConnectionRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
@@ -1716,6 +1759,9 @@ public interface Service {
   /** Response from
    * {@link org.apache.calcite.avatica.remote.Service.OpenConnectionRequest}. */
   class OpenConnectionResponse extends Response {
+    private static final FieldDescriptor METADATA_DESCRIPTOR = Responses.OpenConnectionResponse
+        .getDescriptor().findFieldByNumber(
+            Responses.OpenConnectionResponse.METADATA_FIELD_NUMBER);
     public final RpcMetadataResponse rpcMetadata;
 
     public OpenConnectionResponse() {
@@ -1730,11 +1776,9 @@ public interface Service {
     @Override OpenConnectionResponse deserialize(Message genericMsg) {
       final Responses.OpenConnectionResponse msg = ProtobufService.castProtobufMessage(genericMsg,
           Responses.OpenConnectionResponse.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       RpcMetadataResponse metadata = null;
-      if (ProtobufService.hasField(msg, desc,
-          Responses.OpenConnectionResponse.METADATA_FIELD_NUMBER)) {
+      if (msg.hasField(METADATA_DESCRIPTOR)) {
         metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
       }
 
@@ -1768,6 +1812,9 @@ public interface Service {
   /** Request for
    * {@link Meta#closeConnection(org.apache.calcite.avatica.Meta.ConnectionHandle)}. */
   class CloseConnectionRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.CloseConnectionRequest
+        .getDescriptor().findFieldByNumber(
+            Requests.CloseConnectionRequest.CONNECTION_ID_FIELD_NUMBER);
     public final String connectionId;
 
     CloseConnectionRequest() {
@@ -1787,11 +1834,8 @@ public interface Service {
     @Override CloseConnectionRequest deserialize(Message genericMsg) {
       final Requests.CloseConnectionRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.CloseConnectionRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
-
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.CloseConnectionRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
@@ -1825,6 +1869,10 @@ public interface Service {
   /** Response from
    * {@link org.apache.calcite.avatica.remote.Service.CloseConnectionRequest}. */
   class CloseConnectionResponse extends Response {
+    private static final FieldDescriptor METADATA_DESCRIPTOR = Responses.CloseConnectionResponse
+        .getDescriptor().findFieldByNumber(
+            Responses.CloseConnectionResponse.METADATA_FIELD_NUMBER);
+
     public final RpcMetadataResponse rpcMetadata;
 
     public CloseConnectionResponse() {
@@ -1839,11 +1887,9 @@ public interface Service {
     @Override CloseConnectionResponse deserialize(Message genericMsg) {
       final Responses.CloseConnectionResponse msg = ProtobufService.castProtobufMessage(genericMsg,
           Responses.CloseConnectionResponse.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       RpcMetadataResponse metadata = null;
-      if (ProtobufService.hasField(msg, desc,
-          Responses.CloseConnectionResponse.METADATA_FIELD_NUMBER)) {
+      if (msg.hasField(METADATA_DESCRIPTOR)) {
         metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
       }
 
@@ -1876,6 +1922,12 @@ public interface Service {
 
   /** Request for {@link Meta#connectionSync(Meta.ConnectionHandle, Meta.ConnectionProperties)}. */
   class ConnectionSyncRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.ConnectionSyncRequest
+        .getDescriptor().findFieldByNumber(
+            Requests.ConnectionSyncRequest.CONNECTION_ID_FIELD_NUMBER);
+    private static final FieldDescriptor CONN_PROPS_DESCRIPTOR = Requests.ConnectionSyncRequest
+        .getDescriptor().findFieldByNumber(Requests.ConnectionSyncRequest.CONN_PROPS_FIELD_NUMBER);
+
     public final String connectionId;
     public final Meta.ConnectionProperties connProps;
 
@@ -1899,17 +1951,14 @@ public interface Service {
     @Override ConnectionSyncRequest deserialize(Message genericMsg) {
       final Requests.ConnectionSyncRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.ConnectionSyncRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.ConnectionSyncRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
       Meta.ConnectionProperties connProps = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.ConnectionSyncRequest.CONN_PROPS_FIELD_NUMBER)) {
+      if (msg.hasField(CONN_PROPS_DESCRIPTOR)) {
         connProps = ConnectionPropertiesImpl.fromProto(msg.getConnProps());
       }
 
@@ -1948,6 +1997,8 @@ public interface Service {
   /** Response for
    * {@link Meta#connectionSync(Meta.ConnectionHandle, Meta.ConnectionProperties)}. */
   class ConnectionSyncResponse extends Response {
+    private static final FieldDescriptor METADATA_DESCRIPTOR = Responses.ConnectionSyncResponse
+        .getDescriptor().findFieldByNumber(Responses.ConnectionSyncResponse.METADATA_FIELD_NUMBER);
     public final Meta.ConnectionProperties connProps;
     public final RpcMetadataResponse rpcMetadata;
 
@@ -1966,11 +2017,8 @@ public interface Service {
     @Override ConnectionSyncResponse deserialize(Message genericMsg) {
       final Responses.ConnectionSyncResponse msg = ProtobufService.castProtobufMessage(genericMsg,
           Responses.ConnectionSyncResponse.class);
-      final Descriptor desc = msg.getDescriptorForType();
-
       RpcMetadataResponse metadata = null;
-      if (ProtobufService.hasField(msg, desc,
-          Responses.ConnectionSyncResponse.METADATA_FIELD_NUMBER)) {
+      if (msg.hasField(METADATA_DESCRIPTOR)) {
         metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
       }
 
@@ -2011,6 +2059,9 @@ public interface Service {
   /** Response for
    * {@link Meta#getDatabaseProperties(Meta.ConnectionHandle)}. */
   class DatabasePropertyResponse extends Response {
+    private static final FieldDescriptor METADATA_DESCRIPTOR = Responses.DatabasePropertyResponse
+        .getDescriptor().findFieldByNumber(
+            Responses.DatabasePropertyResponse.METADATA_FIELD_NUMBER);
     public final Map<Meta.DatabaseProperty, Object> map;
     public final RpcMetadataResponse rpcMetadata;
 
@@ -2029,8 +2080,6 @@ public interface Service {
     @Override DatabasePropertyResponse deserialize(Message genericMsg) {
       final Responses.DatabasePropertyResponse msg = ProtobufService.castProtobufMessage(genericMsg,
           Responses.DatabasePropertyResponse.class);
-      final Descriptor desc = msg.getDescriptorForType();
-
       HashMap<Meta.DatabaseProperty, Object> properties = new HashMap<>();
       for (Responses.DatabasePropertyElement property : msg.getPropsList()) {
         final Meta.DatabaseProperty dbProp = Meta.DatabaseProperty.fromProto(property.getKey());
@@ -2067,8 +2116,7 @@ public interface Service {
       }
 
       RpcMetadataResponse metadata = null;
-      if (ProtobufService.hasField(msg, desc,
-          Responses.DatabasePropertyResponse.METADATA_FIELD_NUMBER)) {
+      if (msg.hasField(METADATA_DESCRIPTOR)) {
         metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
       }
 
@@ -2143,7 +2191,20 @@ public interface Service {
    * transport over the wire. Thus, {@link Service#apply} will never return
    * an ErrorResponse.
    */
-  class ErrorResponse extends Response {
+  public class ErrorResponse extends Response {
+    private static final FieldDescriptor ERROR_MESSAGE_DESCRIPTOR = Responses.ErrorResponse
+        .getDescriptor().findFieldByNumber(
+            Responses.ErrorResponse.ERROR_MESSAGE_FIELD_NUMBER);
+    private static final FieldDescriptor SQL_DESCRIPTOR = Responses.ErrorResponse
+        .getDescriptor().findFieldByNumber(
+            Responses.ErrorResponse.SQL_STATE_FIELD_NUMBER);
+    private static final FieldDescriptor SEVERITY_DESCRIPTOR = Responses.ErrorResponse
+        .getDescriptor().findFieldByNumber(
+            Responses.ErrorResponse.SEVERITY_FIELD_NUMBER);
+    private static final FieldDescriptor METADATA_DESCRIPTOR = Responses.ErrorResponse
+        .getDescriptor().findFieldByNumber(
+            Responses.ErrorResponse.METADATA_FIELD_NUMBER);
+
     public static final int UNKNOWN_ERROR_CODE = -1;
     public static final int MISSING_CONNECTION_ERROR_CODE = 1;
 
@@ -2219,30 +2280,28 @@ public interface Service {
     @Override ErrorResponse deserialize(Message genericMsg) {
       final Responses.ErrorResponse msg = ProtobufService.castProtobufMessage(genericMsg,
           Responses.ErrorResponse.class);
-      final Descriptor desc = msg.getDescriptorForType();
-
       List<String> exceptions = null;
       if (msg.getHasExceptions()) {
         exceptions = msg.getExceptionsList();
       }
 
       String errorMessage = null;
-      if (ProtobufService.hasField(msg, desc, Responses.ErrorResponse.ERROR_MESSAGE_FIELD_NUMBER)) {
+      if (msg.hasField(ERROR_MESSAGE_DESCRIPTOR)) {
         errorMessage = msg.getErrorMessage();
       }
 
       String sqlState = null;
-      if (ProtobufService.hasField(msg, desc, Responses.ErrorResponse.SQL_STATE_FIELD_NUMBER)) {
+      if (msg.hasField(SQL_DESCRIPTOR)) {
         sqlState = msg.getSqlState();
       }
 
       AvaticaSeverity severity = null;
-      if (ProtobufService.hasField(msg, desc, Responses.ErrorResponse.SEVERITY_FIELD_NUMBER)) {
+      if (msg.hasField(SEVERITY_DESCRIPTOR)) {
         severity = AvaticaSeverity.fromProto(msg.getSeverity());
       }
 
       RpcMetadataResponse metadata = null;
-      if (ProtobufService.hasField(msg, desc, Responses.ErrorResponse.METADATA_FIELD_NUMBER)) {
+      if (msg.hasField(METADATA_DESCRIPTOR)) {
         metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
       }
 
@@ -2321,6 +2380,14 @@ public interface Service {
    * Request for {@link Service#apply(SyncResultsRequest)}
    */
   class SyncResultsRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.SyncResultsRequest
+        .getDescriptor().findFieldByNumber(Requests.SyncResultsRequest.CONNECTION_ID_FIELD_NUMBER);
+    private static final FieldDescriptor STATEMENT_ID_DESCRIPTOR = Requests.SyncResultsRequest
+        .getDescriptor().findFieldByNumber(Requests.SyncResultsRequest.STATEMENT_ID_FIELD_NUMBER);
+    private static final FieldDescriptor STATE_DESCRIPTOR = Requests.SyncResultsRequest
+        .getDescriptor().findFieldByNumber(Requests.SyncResultsRequest.STATE_FIELD_NUMBER);
+    private static final FieldDescriptor OFFSET_DESCRIPTOR = Requests.SyncResultsRequest
+        .getDescriptor().findFieldByNumber(Requests.SyncResultsRequest.OFFSET_FIELD_NUMBER);
     public final String connectionId;
     public final int statementId;
     public final QueryState state;
@@ -2349,27 +2416,24 @@ public interface Service {
     Request deserialize(Message genericMsg) {
       final Requests.SyncResultsRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.SyncResultsRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.SyncResultsRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
       int statementId = 0;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.SyncResultsRequest.STATEMENT_ID_FIELD_NUMBER)) {
+      if (msg.hasField(STATEMENT_ID_DESCRIPTOR)) {
         statementId = msg.getStatementId();
       }
 
       Common.QueryState state = null;
-      if (ProtobufService.hasField(msg, desc, Requests.SyncResultsRequest.STATE_FIELD_NUMBER)) {
+      if (msg.hasField(STATE_DESCRIPTOR)) {
         state = msg.getState();
       }
 
       long offset = 0;
-      if (ProtobufService.hasField(msg, desc, Requests.SyncResultsRequest.OFFSET_FIELD_NUMBER)) {
+      if (msg.hasField(OFFSET_DESCRIPTOR)) {
         offset = msg.getOffset();
       }
 
@@ -2417,6 +2481,8 @@ public interface Service {
    * Response for {@link Service#apply(SyncResultsRequest)}.
    */
   class SyncResultsResponse extends Response {
+    private static final FieldDescriptor METADATA_DESCRIPTOR = Responses.SyncResultsResponse
+        .getDescriptor().findFieldByNumber(Responses.SyncResultsResponse.METADATA_FIELD_NUMBER);
     public boolean missingStatement = false;
     public final boolean moreResults;
     public final RpcMetadataResponse rpcMetadata;
@@ -2437,11 +2503,9 @@ public interface Service {
     SyncResultsResponse deserialize(Message genericMsg) {
       final Responses.SyncResultsResponse msg = ProtobufService.castProtobufMessage(genericMsg,
           Responses.SyncResultsResponse.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       RpcMetadataResponse metadata = null;
-      if (ProtobufService.hasField(msg, desc,
-          Responses.SyncResultsResponse.METADATA_FIELD_NUMBER)) {
+      if (msg.hasField(METADATA_DESCRIPTOR)) {
         metadata = RpcMetadataResponse.fromProto(msg.getMetadata());
       }
 
@@ -2481,7 +2545,9 @@ public interface Service {
    * This isn't really a "response", but we want to be able to be able to convert it to protobuf
    * and back again, so ignore that there isn't an explicit endpoint for it.
    */
-  class RpcMetadataResponse extends Response {
+  public class RpcMetadataResponse extends Response {
+    private static final FieldDescriptor SERVER_ADDRESS_DESCRIPTOR = Responses.RpcMetadata
+        .getDescriptor().findFieldByNumber(Responses.RpcMetadata.SERVER_ADDRESS_FIELD_NUMBER);
     public final String serverAddress;
 
     public RpcMetadataResponse() {
@@ -2504,10 +2570,8 @@ public interface Service {
     }
 
     static RpcMetadataResponse fromProto(Responses.RpcMetadata msg) {
-      final Descriptor desc = msg.getDescriptorForType();
-
       String serverAddress = null;
-      if (ProtobufService.hasField(msg, desc, Responses.RpcMetadata.SERVER_ADDRESS_FIELD_NUMBER)) {
+      if (msg.hasField(SERVER_ADDRESS_DESCRIPTOR)) {
         serverAddress = msg.getServerAddress();
       }
 
@@ -2531,6 +2595,8 @@ public interface Service {
    * An RPC request to invoke a commit on a Connection.
    */
   class CommitRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.CommitRequest
+        .getDescriptor().findFieldByNumber(Requests.CommitRequest.CONNECTION_ID_FIELD_NUMBER);
     public final String connectionId;
 
     CommitRequest() {
@@ -2548,10 +2614,9 @@ public interface Service {
     @Override CommitRequest deserialize(Message genericMsg) {
       final Requests.CommitRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.CommitRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc, Requests.CommitRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
@@ -2585,6 +2650,9 @@ public interface Service {
    * An RPC response from invoking commit on a Connection.
    */
   class CommitResponse extends Response {
+    private static final CommitResponse INSTANCE = new CommitResponse();
+    private static final Responses.CommitResponse PB_INSTANCE =
+        Responses.CommitResponse.getDefaultInstance();
 
     CommitResponse() {}
 
@@ -2592,11 +2660,11 @@ public interface Service {
       // Checks the type of genericMsg
       ProtobufService.castProtobufMessage(genericMsg, Responses.CommitResponse.class);
 
-      return new CommitResponse();
+      return INSTANCE;
     }
 
     @Override Responses.CommitResponse serialize() {
-      return Responses.CommitResponse.newBuilder().build();
+      return PB_INSTANCE;
     }
 
     @Override public int hashCode() {
@@ -2613,6 +2681,8 @@ public interface Service {
    * An RPC request to invoke a rollback on a Connection.
    */
   class RollbackRequest extends Request {
+    private static final FieldDescriptor CONNECTION_ID_DESCRIPTOR = Requests.RollbackRequest
+        .getDescriptor().findFieldByNumber(Requests.RollbackRequest.CONNECTION_ID_FIELD_NUMBER);
     public final String connectionId;
 
     RollbackRequest() {
@@ -2630,11 +2700,9 @@ public interface Service {
     @Override RollbackRequest deserialize(Message genericMsg) {
       final Requests.RollbackRequest msg = ProtobufService.castProtobufMessage(genericMsg,
           Requests.RollbackRequest.class);
-      final Descriptor desc = msg.getDescriptorForType();
 
       String connectionId = null;
-      if (ProtobufService.hasField(msg, desc,
-          Requests.RollbackRequest.CONNECTION_ID_FIELD_NUMBER)) {
+      if (msg.hasField(CONNECTION_ID_DESCRIPTOR)) {
         connectionId = msg.getConnectionId();
       }
 
@@ -2668,17 +2736,20 @@ public interface Service {
    * An RPC response from invoking rollback on a Connection.
    */
   class RollbackResponse extends Response {
+    private static final RollbackResponse INSTANCE = new RollbackResponse();
+    private static final Responses.RollbackResponse PB_INSTANCE =
+        Responses.RollbackResponse.getDefaultInstance();
 
     RollbackResponse() {}
 
     @Override RollbackResponse deserialize(Message genericMsg) {
       // Check that genericMsg is the expected type
       ProtobufService.castProtobufMessage(genericMsg, Responses.RollbackResponse.class);
-      return new RollbackResponse();
+      return INSTANCE;
     }
 
     @Override Responses.RollbackResponse serialize() {
-      return Responses.RollbackResponse.newBuilder().build();
+      return PB_INSTANCE;
     }
 
     @Override public int hashCode() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/0de38aaa/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java b/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java
index e7c442c..afb15c3 100644
--- a/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java
+++ b/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufHandlerTest.java
@@ -116,8 +116,8 @@ public class ProtobufHandlerTest {
     assertTrue(iter.hasNext());
     Common.ColumnValue column = iter.next();
     assertTrue("The Column should have contained a scalar: " + column,
-        ProtobufService.hasField(column, column.getDescriptorForType(),
-            ColumnValue.SCALAR_VALUE_FIELD_NUMBER));
+        column.hasField(ColumnValue.getDescriptor()
+            .findFieldByNumber(ColumnValue.SCALAR_VALUE_FIELD_NUMBER)));
 
     Common.TypedValue value = column.getScalarValue();
     assertEquals(Common.Rep.BOOLEAN, value.getType());
@@ -126,8 +126,8 @@ public class ProtobufHandlerTest {
     assertTrue(iter.hasNext());
     column = iter.next();
     assertTrue("The Column should have contained a scalar: " + column,
-        ProtobufService.hasField(column, column.getDescriptorForType(),
-            ColumnValue.SCALAR_VALUE_FIELD_NUMBER));
+        column.hasField(ColumnValue.getDescriptor()
+            .findFieldByNumber(ColumnValue.SCALAR_VALUE_FIELD_NUMBER)));
     value = column.getScalarValue();
     assertEquals(Common.Rep.STRING, value.getType());
     assertEquals("my_string", value.getStringValue());


[6/7] calcite git commit: [CALCITE-1118] Add a noop-JDBC driver for testing Avatica server

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/package-info.java
----------------------------------------------------------------------
diff --git a/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/package-info.java b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/package-info.java
new file mode 100644
index 0000000..da1c5c4
--- /dev/null
+++ b/avatica-noop/src/main/java/org/apache/calcite/avatica/noop/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/**
+ * A no-operation implementation of a JDBC driver built for testing Avatica.
+ */
+@PackageMarker
+package org.apache.calcite.avatica.noop;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/avatica-noop/src/main/resources/META-INF/services/java.sql.Driver
----------------------------------------------------------------------
diff --git a/avatica-noop/src/main/resources/META-INF/services/java.sql.Driver b/avatica-noop/src/main/resources/META-INF/services/java.sql.Driver
new file mode 100644
index 0000000..24845b0
--- /dev/null
+++ b/avatica-noop/src/main/resources/META-INF/services/java.sql.Driver
@@ -0,0 +1 @@
+org.apache.calcite.avatica.noop.AvaticaNoopDriver

http://git-wip-us.apache.org/repos/asf/calcite/blob/d27e642c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 25e2806..fecb93e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -128,6 +128,7 @@ limitations under the License.
     <module>avatica</module>
     <module>avatica-metrics</module>
     <module>avatica-metrics-dropwizardmetrics3</module>
+    <module>avatica-noop</module>
     <module>avatica-server</module>
     <module>cassandra</module>
     <module>core</module>
@@ -162,6 +163,11 @@ limitations under the License.
       </dependency>
       <dependency>
         <groupId>org.apache.calcite</groupId>
+        <artifactId>calcite-avatica-noop</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.calcite</groupId>
         <artifactId>calcite-avatica</artifactId>
         <version>${project.version}</version>
         <type>test-jar</type>


[5/7] calcite git commit: [CALCITE-1119] Additional metrics instrumentation for request processing

Posted by el...@apache.org.
[CALCITE-1119] Additional metrics instrumentation for request processing


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

Branch: refs/heads/master
Commit: aecefef8a4420e780737a8f49b72fd146679f2e2
Parents: 1d3a26d
Author: Josh Elser <el...@apache.org>
Authored: Wed Mar 2 17:43:46 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Wed Mar 2 23:26:08 2016 -0500

----------------------------------------------------------------------
 .../calcite/avatica/remote/LocalService.java    | 139 ++++++++++++-------
 1 file changed, 89 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/aecefef8/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 11d15c9..c070ec0 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
@@ -17,25 +17,54 @@
 package org.apache.calcite.avatica.remote;
 
 import org.apache.calcite.avatica.Meta;
+
 import org.apache.calcite.avatica.MetaImpl;
 import org.apache.calcite.avatica.MissingResultsException;
 import org.apache.calcite.avatica.NoSuchStatementException;
+import org.apache.calcite.avatica.metrics.MetricsSystem;
+import org.apache.calcite.avatica.metrics.Timer;
+import org.apache.calcite.avatica.metrics.Timer.Context;
+import org.apache.calcite.avatica.metrics.noop.NoopMetricsSystem;
 
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 
+import static org.apache.calcite.avatica.remote.MetricsHelper.concat;
+
 /**
  * Implementation of {@link Service} that talks to a local {@link Meta}.
  */
 public class LocalService implements Service {
   final Meta meta;
+  final MetricsSystem metrics;
+
+  private final Timer executeTimer;
+  private final Timer commitTimer;
+  private final Timer prepareTimer;
+  private final Timer prepareAndExecuteTimer;
+  private final Timer connectionSyncTimer;
 
   private RpcMetadataResponse serverLevelRpcMetadata;
 
   public LocalService(Meta meta) {
+    this(meta, NoopMetricsSystem.getInstance());
+  }
+
+  public LocalService(Meta meta, MetricsSystem metrics) {
     this.meta = meta;
+    this.metrics = Objects.requireNonNull(metrics);
+
+    this.executeTimer = this.metrics.getTimer(name("Execute"));
+    this.commitTimer = this.metrics.getTimer(name("Commit"));
+    this.prepareTimer = this.metrics.getTimer(name("Prepare"));
+    this.prepareAndExecuteTimer = this.metrics.getTimer(name("PrepareAndExecute"));
+    this.connectionSyncTimer = this.metrics.getTimer(name("ConnectionSync"));
+  }
+
+  private static String name(String timer) {
+    return concat(LocalService.class, timer);
   }
 
   @Override public void setRpcMetadata(RpcMetadataResponse serverLevelRpcMetadata) {
@@ -172,42 +201,46 @@ public class LocalService implements Service {
   }
 
   public PrepareResponse apply(PrepareRequest request) {
-    final Meta.ConnectionHandle ch =
-        new Meta.ConnectionHandle(request.connectionId);
-    final Meta.StatementHandle h =
-        meta.prepare(ch, request.sql, request.maxRowCount);
-    return new PrepareResponse(h, serverLevelRpcMetadata);
+    try (final Context ctx = prepareTimer.start()) {
+      final Meta.ConnectionHandle ch =
+          new Meta.ConnectionHandle(request.connectionId);
+      final Meta.StatementHandle h =
+          meta.prepare(ch, request.sql, request.maxRowCount);
+      return new PrepareResponse(h, serverLevelRpcMetadata);
+    }
   }
 
   public ExecuteResponse apply(PrepareAndExecuteRequest request) {
-    final Meta.StatementHandle sh =
-        new Meta.StatementHandle(request.connectionId, request.statementId, null);
-    try {
-      final Meta.ExecuteResult executeResult =
-          meta.prepareAndExecute(sh, request.sql, request.maxRowCount,
-              new Meta.PrepareCallback() {
-                @Override public Object getMonitor() {
-                  return LocalService.class;
-                }
-
-                @Override public void clear() {
-                }
-
-                @Override public void assign(Meta.Signature signature,
-                    Meta.Frame firstFrame, long updateCount) {
-                }
-
-                @Override public void execute() {
-                }
-              });
-      final List<ResultSetResponse> results = new ArrayList<>();
-      for (Meta.MetaResultSet metaResultSet : executeResult.resultSets) {
-        results.add(toResponse(metaResultSet));
+    try (final Context ctx = prepareAndExecuteTimer.start()) {
+      final Meta.StatementHandle sh =
+          new Meta.StatementHandle(request.connectionId, request.statementId, null);
+      try {
+        final Meta.ExecuteResult executeResult =
+            meta.prepareAndExecute(sh, request.sql, request.maxRowCount,
+                new Meta.PrepareCallback() {
+                  @Override public Object getMonitor() {
+                    return LocalService.class;
+                  }
+
+                  @Override public void clear() {
+                  }
+
+                  @Override public void assign(Meta.Signature signature,
+                      Meta.Frame firstFrame, long updateCount) {
+                  }
+
+                  @Override public void execute() {
+                  }
+                });
+        final List<ResultSetResponse> results = new ArrayList<>();
+        for (Meta.MetaResultSet metaResultSet : executeResult.resultSets) {
+          results.add(toResponse(metaResultSet));
+        }
+        return new ExecuteResponse(results, false, serverLevelRpcMetadata);
+      } catch (NoSuchStatementException e) {
+        // The Statement doesn't exist anymore, bubble up this information
+        return new ExecuteResponse(null, true, serverLevelRpcMetadata);
       }
-      return new ExecuteResponse(results, false, serverLevelRpcMetadata);
-    } catch (NoSuchStatementException e) {
-      // The Statement doesn't exist anymore, bubble up this information
-      return new ExecuteResponse(null, true, serverLevelRpcMetadata);
     }
   }
 
@@ -229,17 +262,19 @@ public class LocalService implements Service {
   }
 
   public ExecuteResponse apply(ExecuteRequest request) {
-    try {
-      final Meta.ExecuteResult executeResult = meta.execute(request.statementHandle,
-          request.parameterValues, request.maxRowCount);
-
-      final List<ResultSetResponse> results = new ArrayList<>(executeResult.resultSets.size());
-      for (Meta.MetaResultSet metaResultSet : executeResult.resultSets) {
-        results.add(toResponse(metaResultSet));
+    try (final Context ctx = executeTimer.start()) {
+      try {
+        final Meta.ExecuteResult executeResult = meta.execute(request.statementHandle,
+            request.parameterValues, request.maxRowCount);
+
+        final List<ResultSetResponse> results = new ArrayList<>(executeResult.resultSets.size());
+        for (Meta.MetaResultSet metaResultSet : executeResult.resultSets) {
+          results.add(toResponse(metaResultSet));
+        }
+        return new ExecuteResponse(results, false, serverLevelRpcMetadata);
+      } catch (NoSuchStatementException e) {
+        return new ExecuteResponse(null, true, serverLevelRpcMetadata);
       }
-      return new ExecuteResponse(results, false, serverLevelRpcMetadata);
-    } catch (NoSuchStatementException e) {
-      return new ExecuteResponse(null, true, serverLevelRpcMetadata);
     }
   }
 
@@ -272,11 +307,13 @@ public class LocalService implements Service {
   }
 
   public ConnectionSyncResponse apply(ConnectionSyncRequest request) {
-    final Meta.ConnectionHandle ch =
-        new Meta.ConnectionHandle(request.connectionId);
-    final Meta.ConnectionProperties connProps =
-        meta.connectionSync(ch, request.connProps);
-    return new ConnectionSyncResponse(connProps, serverLevelRpcMetadata);
+    try (final Context ctx = connectionSyncTimer.start()) {
+      final Meta.ConnectionHandle ch =
+          new Meta.ConnectionHandle(request.connectionId);
+      final Meta.ConnectionProperties connProps =
+          meta.connectionSync(ch, request.connProps);
+      return new ConnectionSyncResponse(connProps, serverLevelRpcMetadata);
+    }
   }
 
   public DatabasePropertyResponse apply(DatabasePropertyRequest request) {
@@ -302,10 +339,12 @@ public class LocalService implements Service {
   }
 
   public CommitResponse apply(CommitRequest request) {
-    meta.commit(new Meta.ConnectionHandle(request.connectionId));
+    try (final Context ctx = commitTimer.start()) {
+      meta.commit(new Meta.ConnectionHandle(request.connectionId));
 
-    // If commit() errors, let the ErrorResponse be sent back via an uncaught Exception.
-    return new CommitResponse();
+      // If commit() errors, let the ErrorResponse be sent back via an uncaught Exception.
+      return new CommitResponse();
+    }
   }
 
   public RollbackResponse apply(RollbackRequest request) {


[4/7] calcite git commit: [CALCITE-1094] Replace ByteArrayOutputStream to avoid synchronized writes

Posted by el...@apache.org.
[CALCITE-1094] Replace ByteArrayOutputStream to avoid synchronized writes

Pull in the ZeroCopyByteString class. We can cap the amount of byte[]'s
that we are making by providing a buffer to the current thread. This also
avoids any synchronization on a typical object pool.


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

Branch: refs/heads/master
Commit: 1d3a26dfac17fea458402a637449007dc095bced
Parents: 0de38aa
Author: Josh Elser <el...@apache.org>
Authored: Wed Mar 2 17:43:41 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Wed Mar 2 23:26:02 2016 -0500

----------------------------------------------------------------------
 .../avatica/server/AvaticaJsonHandler.java      |  16 +-
 .../avatica/server/AvaticaProtobufHandler.java  |  20 ++-
 .../protobuf/HBaseZeroCopyByteString.java       |  77 ++++++++++
 .../java/com/google/protobuf/package-info.java  |  26 ++++
 .../apache/calcite/avatica/AvaticaUtils.java    |  47 ++++--
 .../avatica/remote/ProtobufTranslation.java     |  10 +-
 .../avatica/remote/ProtobufTranslationImpl.java | 146 +++++++++++++++---
 .../avatica/remote/RequestTranslator.java       |   3 +-
 .../apache/calcite/avatica/remote/Service.java  |   8 +-
 .../calcite/avatica/remote/TypedValue.java      |   3 +-
 .../avatica/util/UnsynchronizedBuffer.java      | 152 +++++++++++++++++++
 .../remote/ProtobufSerializationTest.java       |  75 +++++++++
 .../avatica/util/UnsynchronizedBufferTest.java  |  41 +++++
 13 files changed, 578 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaJsonHandler.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaJsonHandler.java b/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaJsonHandler.java
index 703a2c3..34a9333 100644
--- a/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaJsonHandler.java
+++ b/avatica-server/src/main/java/org/apache/calcite/avatica/server/AvaticaJsonHandler.java
@@ -25,6 +25,7 @@ import org.apache.calcite.avatica.remote.Handler.HandlerResponse;
 import org.apache.calcite.avatica.remote.JsonHandler;
 import org.apache.calcite.avatica.remote.Service;
 import org.apache.calcite.avatica.remote.Service.RpcMetadataResponse;
+import org.apache.calcite.avatica.util.UnsynchronizedBuffer;
 
 import org.eclipse.jetty.server.Request;
 import org.eclipse.jetty.server.handler.AbstractHandler;
@@ -54,6 +55,8 @@ public class AvaticaJsonHandler extends AbstractHandler implements MetricsAwareA
   final MetricsSystem metrics;
   final Timer requestTimer;
 
+  final ThreadLocal<UnsynchronizedBuffer> threadLocalBuffer;
+
   public AvaticaJsonHandler(Service service) {
     this(service, NoopMetricsSystem.getInstance());
   }
@@ -67,6 +70,12 @@ public class AvaticaJsonHandler extends AbstractHandler implements MetricsAwareA
     // Metrics
     this.requestTimer = this.metrics.getTimer(
         concat(AvaticaJsonHandler.class, MetricsAwareAvaticaHandler.REQUEST_TIMER_NAME));
+
+    this.threadLocalBuffer = new ThreadLocal<UnsynchronizedBuffer>() {
+      @Override public UnsynchronizedBuffer initialValue() {
+        return new UnsynchronizedBuffer();
+      }
+    };
   }
 
   public void handle(String target, Request baseRequest,
@@ -80,8 +89,13 @@ public class AvaticaJsonHandler extends AbstractHandler implements MetricsAwareA
         // The latter allows very large requests without hitting HTTP 413.
         String rawRequest = request.getHeader("request");
         if (rawRequest == null) {
+          // Avoid a new buffer creation for every HTTP request
+          final UnsynchronizedBuffer buffer = threadLocalBuffer.get();
           try (ServletInputStream inputStream = request.getInputStream()) {
-            rawRequest = AvaticaUtils.readFully(inputStream);
+            rawRequest = AvaticaUtils.readFully(inputStream, buffer);
+          } finally {
+            // Reset the offset into the buffer after we're done
+            buffer.reset();
           }
         }
         final String jsonRequest =

http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/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 aeebad7..27e73de 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
@@ -28,6 +28,7 @@ import org.apache.calcite.avatica.remote.ProtobufTranslation;
 import org.apache.calcite.avatica.remote.ProtobufTranslationImpl;
 import org.apache.calcite.avatica.remote.Service;
 import org.apache.calcite.avatica.remote.Service.RpcMetadataResponse;
+import org.apache.calcite.avatica.util.UnsynchronizedBuffer;
 
 import org.eclipse.jetty.server.Request;
 import org.eclipse.jetty.server.handler.AbstractHandler;
@@ -55,19 +56,28 @@ public class AvaticaProtobufHandler extends AbstractHandler implements MetricsAw
   private final MetricsSystem metrics;
   private final Timer requestTimer;
 
+  final ThreadLocal<UnsynchronizedBuffer> threadLocalBuffer;
+
   public AvaticaProtobufHandler(Service service) {
     this(service, NoopMetricsSystem.getInstance());
   }
 
   public AvaticaProtobufHandler(Service service, MetricsSystem metrics) {
-    this.protobufTranslation = new ProtobufTranslationImpl();
     this.service = Objects.requireNonNull(service);
     this.metrics = Objects.requireNonNull(metrics);
-    this.pbHandler = new ProtobufHandler(service, protobufTranslation, metrics);
 
     this.requestTimer = this.metrics.getTimer(
         MetricsHelper.concat(AvaticaProtobufHandler.class,
             MetricsAwareAvaticaHandler.REQUEST_TIMER_NAME));
+
+    this.protobufTranslation = new ProtobufTranslationImpl();
+    this.pbHandler = new ProtobufHandler(service, protobufTranslation, metrics);
+
+    this.threadLocalBuffer = new ThreadLocal<UnsynchronizedBuffer>() {
+      @Override public UnsynchronizedBuffer initialValue() {
+        return new UnsynchronizedBuffer();
+      }
+    };
   }
 
   public void handle(String target, Request baseRequest,
@@ -78,8 +88,12 @@ public class AvaticaProtobufHandler extends AbstractHandler implements MetricsAw
       response.setStatus(HttpServletResponse.SC_OK);
       if (request.getMethod().equals("POST")) {
         byte[] requestBytes;
+        // Avoid a new buffer creation for every HTTP request
+        final UnsynchronizedBuffer buffer = threadLocalBuffer.get();
         try (ServletInputStream inputStream = request.getInputStream()) {
-          requestBytes = AvaticaUtils.readFullyToBytes(inputStream);
+          requestBytes = AvaticaUtils.readFullyToBytes(inputStream, buffer);
+        } finally {
+          buffer.reset();
         }
 
         HandlerResponse<byte[]> handlerResponse = pbHandler.apply(requestBytes);

http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/avatica/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java b/avatica/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java
new file mode 100644
index 0000000..62c4dd2
--- /dev/null
+++ b/avatica/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java
@@ -0,0 +1,77 @@
+/*
+ * 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 com.google.protobuf;
+
+/**
+ * Helper class to extract byte arrays from {@link ByteString} without copy.
+ *
+ * Without this protobufs would force us to copy every single byte array out
+ * of the objects de-serialized from the wire (which already do one copy, on
+ * top of the copies the JVM does to go from kernel buffer to C buffer and
+ * from C buffer to JVM buffer).
+ *
+ * Graciously copied from Apache HBase.
+ */
+public final class HBaseZeroCopyByteString extends LiteralByteString {
+  // Gotten from AsyncHBase code base with permission.
+  /** Private constructor so this class cannot be instantiated. */
+  private HBaseZeroCopyByteString() {
+    super(null);
+    throw new UnsupportedOperationException("Should never be here.");
+  }
+
+  /**
+   * Wraps a byte array in a {@link ByteString} without copying it.
+   *
+   * @param array The byte array to wrap
+   * @return a ByteString wrapping the <code>array</code>
+   */
+  public static ByteString wrap(final byte[] array) {
+    return new LiteralByteString(array);
+  }
+
+  /**
+   * Wraps a subset of a byte array in a {@link ByteString} without copying it.
+   *
+   * @param array The byte array to wrap
+   * @param offset the start of data in the array
+   * @param length The number of bytes of data at <code>offset</code>
+   * @return a ByteString wrapping the <code>array</code>
+   */
+  public static ByteString wrap(final byte[] array, int offset, int length) {
+    return new BoundedByteString(array, offset, length);
+  }
+
+
+  /**
+   * Extracts the byte array from the given {@link ByteString} without copy.
+   * @param buf A buffer from which to extract the array.  This buffer must be
+   * actually an instance of a {@code LiteralByteString}.
+   *
+   * @param buf <code>ByteString</code> to access
+   * @return The underlying byte array of the ByteString
+   */
+  public static byte[] zeroCopyGetBytes(final ByteString buf) {
+    if (buf instanceof LiteralByteString) {
+      return ((LiteralByteString) buf).bytes;
+    }
+    throw new UnsupportedOperationException("Need a LiteralByteString, got a "
+                                            + buf.getClass().getName());
+  }
+}
+
+// End HBaseZeroCopyByteString.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/avatica/src/main/java/com/google/protobuf/package-info.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/com/google/protobuf/package-info.java b/avatica/src/main/java/com/google/protobuf/package-info.java
new file mode 100644
index 0000000..92f110e
--- /dev/null
+++ b/avatica/src/main/java/com/google/protobuf/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Avatica-custom classes to access protected classes in Google Protobuf.
+ */
+@PackageMarker
+package com.google.protobuf;
+
+import org.apache.calcite.avatica.util.PackageMarker;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/avatica/src/main/java/org/apache/calcite/avatica/AvaticaUtils.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaUtils.java b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaUtils.java
index 9382f87..a999f19 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/AvaticaUtils.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/AvaticaUtils.java
@@ -16,13 +16,15 @@
  */
 package org.apache.calcite.avatica;
 
-import java.io.ByteArrayOutputStream;
+import org.apache.calcite.avatica.util.UnsynchronizedBuffer;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.lang.invoke.MethodHandle;
 import java.lang.invoke.MethodHandles;
 import java.lang.invoke.MethodType;
 import java.lang.reflect.Field;
+import java.nio.charset.StandardCharsets;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.AbstractList;
@@ -45,6 +47,12 @@ public class AvaticaUtils {
 
   private static final Set<String> UNIQUE_STRINGS = new HashSet<>();
 
+  private static final ThreadLocal<byte[]> PER_THREAD_BUFFER  = new ThreadLocal<byte[]>() {
+    @Override protected byte[] initialValue() {
+      return new byte[4096];
+    }
+  };
+
   private AvaticaUtils() {}
 
   static {
@@ -200,25 +208,46 @@ public class AvaticaUtils {
 
   /** Reads the contents of an input stream and returns as a string. */
   public static String readFully(InputStream inputStream) throws IOException {
-    return _readFully(inputStream).toString();
+    return readFully(inputStream, new UnsynchronizedBuffer(1024));
   }
 
+  /** Reads the contents of an input stream and returns as a string. */
+  public static String readFully(InputStream inputStream, UnsynchronizedBuffer buffer)
+      throws IOException {
+    // Variant that lets us use a pooled Buffer
+    final byte[] bytes = _readFully(inputStream, buffer);
+    return new String(bytes, 0, bytes.length, StandardCharsets.UTF_8);
+  }
+
+  /** Reads the contents of an input stream and returns as a string. */
   public static byte[] readFullyToBytes(InputStream inputStream) throws IOException {
-    return _readFully(inputStream).toByteArray();
+    return readFullyToBytes(inputStream, new UnsynchronizedBuffer(1024));
   }
 
-  /** Reads the contents of an input stream and returns a ByteArrayOutputStrema. */
-  static ByteArrayOutputStream _readFully(InputStream inputStream) throws IOException {
-    final byte[] bytes = new byte[4096];
-    final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+  /** Reads the contents of an input stream and returns as a string. */
+  public static byte[] readFullyToBytes(InputStream inputStream, UnsynchronizedBuffer buffer)
+      throws IOException {
+    // Variant that lets us use a pooled Buffer
+    return _readFully(inputStream, buffer);
+  }
+
+  /**
+   * Reads the contents of an input stream and returns a byte array.
+   *
+   * @param inputStream the input to read from.
+   * @return A byte array whose length is equal to the number of bytes contained.
+   */
+  static byte[] _readFully(InputStream inputStream, UnsynchronizedBuffer buffer)
+      throws IOException {
+    final byte[] bytes = PER_THREAD_BUFFER.get();
     for (;;) {
       int count = inputStream.read(bytes, 0, bytes.length);
       if (count < 0) {
         break;
       }
-      baos.write(bytes, 0, count);
+      buffer.write(bytes, 0, count);
     }
-    return baos;
+    return buffer.toArray();
   }
 
   /** Invokes {@code Statement#setLargeMaxRows}, falling back on

http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslation.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslation.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslation.java
index acb82db..7142d59 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslation.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslation.java
@@ -19,8 +19,6 @@ package org.apache.calcite.avatica.remote;
 import org.apache.calcite.avatica.remote.Service.Request;
 import org.apache.calcite.avatica.remote.Service.Response;
 
-import com.google.protobuf.InvalidProtocolBufferException;
-
 import java.io.IOException;
 
 /**
@@ -49,18 +47,18 @@ public interface ProtobufTranslation {
    *
    * @param bytes Serialized protocol buffer request from client
    * @return A Request object for the given bytes
-   * @throws InvalidProtocolBufferException If the protocol buffer cannot be deserialized
+   * @throws IOException If the protocol buffer cannot be deserialized
    */
-  Request parseRequest(byte[] bytes) throws InvalidProtocolBufferException;
+  Request parseRequest(byte[] bytes) throws IOException;
 
   /**
    * Parses a serialized protocol buffer response into a {@link Response}.
    *
    * @param bytes Serialized protocol buffer request from server
    * @return The Response object for the given bytes
-   * @throws InvalidProtocolBufferException If the protocol buffer cannot be deserialized
+   * @throws IOException If the protocol buffer cannot be deserialized
    */
-  Response parseResponse(byte[] bytes) throws InvalidProtocolBufferException;
+  Response parseResponse(byte[] bytes) throws IOException;
 }
 
 // End ProtobufTranslation.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslationImpl.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslationImpl.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslationImpl.java
index 646d706..80d2b22 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslationImpl.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/ProtobufTranslationImpl.java
@@ -54,17 +54,22 @@ import org.apache.calcite.avatica.proto.Responses.SyncResultsResponse;
 import org.apache.calcite.avatica.remote.Service.Request;
 import org.apache.calcite.avatica.remote.Service.Response;
 import org.apache.calcite.avatica.remote.Service.RpcMetadataResponse;
+import org.apache.calcite.avatica.util.UnsynchronizedBuffer;
 
 import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.HBaseZeroCopyByteString;
 import com.google.protobuf.Message;
 
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
 
 /**
  * Implementation of {@link ProtobufTranslationImpl} that translates
@@ -75,9 +80,10 @@ public class ProtobufTranslationImpl implements ProtobufTranslation {
   // Extremely ugly mapping of PB class name into a means to convert it to the POJO
   private static final Map<String, RequestTranslator> REQUEST_PARSERS;
   private static final Map<String, ResponseTranslator> RESPONSE_PARSERS;
+  private static final Map<Class<?>, ByteString> MESSAGE_CLASSES;
 
   static {
-    HashMap<String, RequestTranslator> reqParsers = new HashMap<>();
+    Map<String, RequestTranslator> reqParsers = new ConcurrentHashMap<>();
     reqParsers.put(CatalogsRequest.class.getName(),
         new RequestTranslator(CatalogsRequest.parser(), new Service.CatalogsRequest()));
     reqParsers.put(OpenConnectionRequest.class.getName(),
@@ -123,7 +129,7 @@ public class ProtobufTranslationImpl implements ProtobufTranslation {
 
     REQUEST_PARSERS = Collections.unmodifiableMap(reqParsers);
 
-    HashMap<String, ResponseTranslator> respParsers = new HashMap<>();
+    Map<String, ResponseTranslator> respParsers = new ConcurrentHashMap<>();
     respParsers.put(OpenConnectionResponse.class.getName(),
         new ResponseTranslator(OpenConnectionResponse.parser(),
             new Service.OpenConnectionResponse()));
@@ -162,8 +168,65 @@ public class ProtobufTranslationImpl implements ProtobufTranslation {
         new ResponseTranslator(RollbackResponse.parser(), new Service.RollbackResponse()));
 
     RESPONSE_PARSERS = Collections.unmodifiableMap(respParsers);
+
+    Map<Class<?>, ByteString> messageClassNames = new ConcurrentHashMap<>();
+    for (Class<?> msgClz : getAllMessageClasses()) {
+      messageClassNames.put(msgClz, wrapClassName(msgClz));
+    }
+    MESSAGE_CLASSES = Collections.unmodifiableMap(messageClassNames);
+  }
+
+  private static List<Class<?>> getAllMessageClasses() {
+    List<Class<?>> messageClasses = new ArrayList<>();
+    messageClasses.add(CatalogsRequest.class);
+    messageClasses.add(CloseConnectionRequest.class);
+    messageClasses.add(CloseStatementRequest.class);
+    messageClasses.add(ColumnsRequest.class);
+    messageClasses.add(CommitRequest.class);
+    messageClasses.add(ConnectionSyncRequest.class);
+    messageClasses.add(CreateStatementRequest.class);
+    messageClasses.add(DatabasePropertyRequest.class);
+    messageClasses.add(ExecuteRequest.class);
+    messageClasses.add(FetchRequest.class);
+    messageClasses.add(OpenConnectionRequest.class);
+    messageClasses.add(PrepareAndExecuteRequest.class);
+    messageClasses.add(PrepareRequest.class);
+    messageClasses.add(RollbackRequest.class);
+    messageClasses.add(SchemasRequest.class);
+    messageClasses.add(SyncResultsRequest.class);
+    messageClasses.add(TableTypesRequest.class);
+    messageClasses.add(TablesRequest.class);
+    messageClasses.add(TypeInfoRequest.class);
+    messageClasses.add(CloseConnectionResponse.class);
+    messageClasses.add(CloseStatementResponse.class);
+    messageClasses.add(CommitResponse.class);
+    messageClasses.add(ConnectionSyncResponse.class);
+    messageClasses.add(CreateStatementResponse.class);
+    messageClasses.add(DatabasePropertyResponse.class);
+    messageClasses.add(ErrorResponse.class);
+    messageClasses.add(ExecuteResponse.class);
+    messageClasses.add(FetchResponse.class);
+    messageClasses.add(OpenConnectionResponse.class);
+    messageClasses.add(PrepareResponse.class);
+    messageClasses.add(ResultSetResponse.class);
+    messageClasses.add(RollbackResponse.class);
+    messageClasses.add(RpcMetadata.class);
+    messageClasses.add(SyncResultsResponse.class);
+
+    return messageClasses;
+  }
+
+  private static ByteString wrapClassName(Class<?> clz) {
+    return HBaseZeroCopyByteString.wrap(clz.getName().getBytes(UTF_8));
   }
 
+  private final ThreadLocal<UnsynchronizedBuffer> threadLocalBuffer =
+      new ThreadLocal<UnsynchronizedBuffer>() {
+        @Override protected UnsynchronizedBuffer initialValue() {
+          return new UnsynchronizedBuffer();
+        }
+      };
+
   /**
    * Fetches the concrete message's Parser implementation.
    *
@@ -207,42 +270,79 @@ public class ProtobufTranslationImpl implements ProtobufTranslation {
   }
 
   @Override public byte[] serializeResponse(Response response) throws IOException {
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    Message responseMsg = response.serialize();
-    serializeMessage(out, responseMsg);
-    return out.toByteArray();
+    // Avoid BAOS for its synchronized write methods, we don't need that concurrency control
+    UnsynchronizedBuffer out = threadLocalBuffer.get();
+    try {
+      Message responseMsg = response.serialize();
+      serializeMessage(out, responseMsg);
+      return out.toArray();
+    } finally {
+      out.reset();
+    }
   }
 
   @Override public byte[] serializeRequest(Request request) throws IOException {
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    Message requestMsg = request.serialize();
-    serializeMessage(out, requestMsg);
-    return out.toByteArray();
+    // Avoid BAOS for its synchronized write methods, we don't need that concurrency control
+    UnsynchronizedBuffer out = threadLocalBuffer.get();
+    try {
+      Message requestMsg = request.serialize();
+      serializeMessage(out, requestMsg);
+      return out.toArray();
+    } finally {
+      out.reset();
+    }
   }
 
   void serializeMessage(OutputStream out, Message msg) throws IOException {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    msg.writeTo(baos);
+    // Serialize the protobuf message
+    UnsynchronizedBuffer buffer = threadLocalBuffer.get();
+    ByteString serializedMsg;
+    try {
+      msg.writeTo(buffer);
+      // Make a bytestring from it
+      serializedMsg = HBaseZeroCopyByteString.wrap(buffer.toArray());
+    } finally {
+      buffer.reset();
+    }
 
-    // TODO Using ByteString is copying the bytes of the message which sucks. Could try to
-    // lift the ZeroCopy implementation from HBase.
-    WireMessage wireMsg = WireMessage.newBuilder().setName(msg.getClass().getName()).
-        setWrappedMessage(ByteString.copyFrom(baos.toByteArray())).build();
+    // Wrap the serialized message in a WireMessage
+    WireMessage wireMsg = WireMessage.newBuilder().setNameBytes(getClassNameBytes(msg.getClass()))
+        .setWrappedMessage(serializedMsg).build();
 
+    // Write the WireMessage to the provided OutputStream
     wireMsg.writeTo(out);
   }
 
-  @Override public Request parseRequest(byte[] bytes) throws InvalidProtocolBufferException {
-    WireMessage wireMsg = WireMessage.parseFrom(bytes);
+  ByteString getClassNameBytes(Class<?> clz) {
+    ByteString byteString = MESSAGE_CLASSES.get(clz);
+    if (null == byteString) {
+      throw new IllegalArgumentException("Missing ByteString for " + clz.getName());
+    }
+    return byteString;
+  }
+
+  @Override public Request parseRequest(byte[] bytes) throws IOException {
+    ByteString byteString = HBaseZeroCopyByteString.wrap(bytes);
+    CodedInputStream inputStream = byteString.newCodedInput();
+    // Enable aliasing to avoid an extra copy to get at the serialized Request inside of the
+    // WireMessage.
+    inputStream.enableAliasing(true);
+    WireMessage wireMsg = WireMessage.parseFrom(inputStream);
 
     String serializedMessageClassName = wireMsg.getName();
     RequestTranslator translator = getParserForRequest(serializedMessageClassName);
 
+    // The ByteString should be logical offsets into the original byte array
     return translator.transform(wireMsg.getWrappedMessage());
   }
 
-  @Override public Response parseResponse(byte[] bytes) throws InvalidProtocolBufferException {
-    WireMessage wireMsg = WireMessage.parseFrom(bytes);
+  @Override public Response parseResponse(byte[] bytes) throws IOException {
+    ByteString byteString = HBaseZeroCopyByteString.wrap(bytes);
+    CodedInputStream inputStream = byteString.newCodedInput();
+    // Enable aliasing to avoid an extra copy to get at the serialized Response inside of the
+    // WireMessage.
+    inputStream.enableAliasing(true);
+    WireMessage wireMsg = WireMessage.parseFrom(inputStream);
 
     String serializedMessageClassName = wireMsg.getName();
     ResponseTranslator translator = getParserForResponse(serializedMessageClassName);

http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/avatica/src/main/java/org/apache/calcite/avatica/remote/RequestTranslator.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/RequestTranslator.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/RequestTranslator.java
index 0dadc78..417c6ed 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/RequestTranslator.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/RequestTranslator.java
@@ -36,7 +36,8 @@ public class RequestTranslator {
 
   public Service.Request transform(ByteString serializedMessage) throws
       InvalidProtocolBufferException {
-    Message msg = parser.parseFrom(serializedMessage);
+    // This should already be an aliased CodedInputStream from the WireMessage parsing.
+    Message msg = parser.parseFrom(serializedMessage.newCodedInput());
     return impl.deserialize(msg);
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/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 aee5b29..5790848 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
@@ -31,8 +31,9 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
-
+import com.google.protobuf.ByteString;
 import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.HBaseZeroCopyByteString;
 import com.google.protobuf.Message;
 
 import java.io.PrintWriter;
@@ -2549,13 +2550,16 @@ public interface Service {
     private static final FieldDescriptor SERVER_ADDRESS_DESCRIPTOR = Responses.RpcMetadata
         .getDescriptor().findFieldByNumber(Responses.RpcMetadata.SERVER_ADDRESS_FIELD_NUMBER);
     public final String serverAddress;
+    private final ByteString serverAddressAsBytes;
 
     public RpcMetadataResponse() {
       this.serverAddress = null;
+      this.serverAddressAsBytes = null;
     }
 
     public RpcMetadataResponse(@JsonProperty("serverAddress") String serverAddress) {
       this.serverAddress = serverAddress;
+      this.serverAddressAsBytes = HBaseZeroCopyByteString.wrap(serverAddress.getBytes());
     }
 
     @Override RpcMetadataResponse deserialize(Message genericMsg) {
@@ -2566,7 +2570,7 @@ public interface Service {
     }
 
     @Override Responses.RpcMetadata serialize() {
-      return Responses.RpcMetadata.newBuilder().setServerAddress(serverAddress).build();
+      return Responses.RpcMetadata.newBuilder().setServerAddressBytes(serverAddressAsBytes).build();
     }
 
     static RpcMetadataResponse fromProto(Responses.RpcMetadata msg) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/avatica/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
index 5c80816..d96293b 100644
--- a/avatica/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
@@ -23,6 +23,7 @@ import org.apache.calcite.avatica.util.DateTimeUtils;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.protobuf.HBaseZeroCopyByteString;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
@@ -345,7 +346,7 @@ public class TypedValue {
       break;
     case BYTE_STRING:
     case STRING:
-      builder.setStringValue((String) value);
+      builder.setStringValueBytes(HBaseZeroCopyByteString.wrap(((String) value).getBytes()));
       break;
     case PRIMITIVE_CHAR:
     case CHARACTER:

http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/avatica/src/main/java/org/apache/calcite/avatica/util/UnsynchronizedBuffer.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/util/UnsynchronizedBuffer.java b/avatica/src/main/java/org/apache/calcite/avatica/util/UnsynchronizedBuffer.java
new file mode 100644
index 0000000..8daee60
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/util/UnsynchronizedBuffer.java
@@ -0,0 +1,152 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A utility class for reading and writing bytes to byte buffers without synchronization. A
+ * reduced variant taken from Apache Accumulo. This class is <b>not</b> thread-safe by design.
+ * It is up to the caller to guarantee mutual exclusion as necessary.
+ */
+public class UnsynchronizedBuffer extends OutputStream {
+  // Anything larger than 64K, reap the backing buffer
+  private static final int LARGE_BUFFER_SIZE = 1024 * 64;
+
+  final int initialCapacity;
+  int offset = 0;
+  byte[] data;
+
+  /**
+   * Creates a new writer.
+   */
+  public UnsynchronizedBuffer() {
+    this(4096);
+  }
+
+  /**
+   * Creates a new writer.
+   *
+   * @param initialCapacity initial byte capacity
+   */
+  public UnsynchronizedBuffer(int initialCapacity) {
+    this.initialCapacity = initialCapacity;
+    data = new byte[initialCapacity];
+  }
+
+  private void reserve(int l) {
+    if (offset + l > data.length) {
+      int newSize = UnsynchronizedBuffer.nextArraySize(offset + l);
+
+      byte[] newData = new byte[newSize];
+      System.arraycopy(data, 0, newData, 0, offset);
+      data = newData;
+    }
+
+  }
+
+  /**
+   * Adds bytes to this writer's buffer.
+   *
+   * @param bytes byte array
+   * @param off offset into array to start copying bytes
+   * @param length number of bytes to add
+   * @throws IndexOutOfBoundsException if off or length are invalid
+   */
+  public void write(byte[] bytes, int off, int length) {
+    reserve(length);
+    System.arraycopy(bytes, off, data, offset, length);
+    offset += length;
+  }
+
+  @Override public void write(int b) throws IOException {
+    reserve(1);
+    data[offset] = (byte) b;
+    offset++;
+  }
+
+  /**
+   * Gets (a copy of) the contents of this writer's buffer.
+   *
+   * @return byte buffer contents
+   */
+  public byte[] toArray() {
+    byte[] ret = new byte[offset];
+    System.arraycopy(data, 0, ret, 0, offset);
+    return ret;
+  }
+
+  /**
+   * Resets the internal pointer into the buffer.
+   */
+  public void reset() {
+    offset = 0;
+    if (data.length >= LARGE_BUFFER_SIZE) {
+      data = new byte[this.initialCapacity];
+    }
+  }
+
+  /**
+   * @return The current offset into the backing array.
+   */
+  public int getOffset() {
+    return offset;
+  }
+
+  /**
+   * @return The current length of the backing array.
+   */
+  public long getSize() {
+    return data.length;
+  }
+
+  /**
+   * Determines what next array size should be by rounding up to next power of two.
+   *
+   * @param i current array size
+   * @return next array size
+   * @throws IllegalArgumentException if i is negative
+   */
+  public static int nextArraySize(int i) {
+    if (i < 0) {
+      throw new IllegalArgumentException();
+    }
+
+    if (i > (1 << 30)) {
+      return Integer.MAX_VALUE; // this is the next power of 2 minus one... a special case
+    }
+
+    if (i == 0) {
+      return 1;
+    }
+
+    // round up to next power of two
+    int ret = i;
+    ret--;
+    ret |= ret >> 1;
+    ret |= ret >> 2;
+    ret |= ret >> 4;
+    ret |= ret >> 8;
+    ret |= ret >> 16;
+    ret++;
+
+    return ret;
+  }
+}
+
+// End UnsynchronizedBuffer.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufSerializationTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufSerializationTest.java b/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufSerializationTest.java
new file mode 100644
index 0000000..cd8a329
--- /dev/null
+++ b/avatica/src/test/java/org/apache/calcite/avatica/remote/ProtobufSerializationTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.ColumnMetaData.Rep;
+import org.apache.calcite.avatica.Meta.Signature;
+import org.apache.calcite.avatica.Meta.StatementHandle;
+import org.apache.calcite.avatica.proto.Common.WireMessage;
+import org.apache.calcite.avatica.proto.Requests;
+import org.apache.calcite.avatica.remote.Service.Request;
+
+import com.google.protobuf.HBaseZeroCopyByteString;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Protobuf serialization tests.
+ */
+public class ProtobufSerializationTest {
+
+  private Signature getSignature() {
+    return null;
+  }
+
+  private List<TypedValue> getTypedValues() {
+    List<TypedValue> paramValues =
+        Arrays.asList(TypedValue.create(Rep.BOOLEAN.name(), Boolean.TRUE),
+            TypedValue.create(Rep.STRING.name(), "string"));
+    return paramValues;
+  }
+
+  @Test public void testExecuteSerialization() throws Exception {
+    Service.ExecuteRequest executeRequest = new Service.ExecuteRequest(
+        new StatementHandle("connection", 12345, getSignature()), getTypedValues(), 0);
+
+    Requests.ExecuteRequest pbExecuteRequest = executeRequest.serialize();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream(1024);
+    pbExecuteRequest.writeTo(baos);
+
+    byte[] serialized = baos.toByteArray();
+    baos.reset();
+    WireMessage wireMsg = WireMessage.newBuilder().setName(Requests.ExecuteRequest.class.getName())
+        .setWrappedMessage(HBaseZeroCopyByteString.wrap(serialized)).build();
+    wireMsg.writeTo(baos);
+    serialized = baos.toByteArray();
+
+    ProtobufTranslation translator = new ProtobufTranslationImpl();
+
+    Request newRequest = translator.parseRequest(serialized);
+
+    Assert.assertEquals(executeRequest, newRequest);
+  }
+
+}
+
+// End ProtobufSerializationTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/1d3a26df/avatica/src/test/java/org/apache/calcite/avatica/util/UnsynchronizedBufferTest.java
----------------------------------------------------------------------
diff --git a/avatica/src/test/java/org/apache/calcite/avatica/util/UnsynchronizedBufferTest.java b/avatica/src/test/java/org/apache/calcite/avatica/util/UnsynchronizedBufferTest.java
new file mode 100644
index 0000000..a448d3e
--- /dev/null
+++ b/avatica/src/test/java/org/apache/calcite/avatica/util/UnsynchronizedBufferTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.util;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for the UnsynchronizedBuffer.
+ */
+public class UnsynchronizedBufferTest {
+
+  @Test public void testArrayResizing() {
+    int size = 64;
+    int expected = 128;
+    for (int i = 0; i < 10; i++) {
+      // We keep being one byte short to contain this message
+      int next = UnsynchronizedBuffer.nextArraySize(size + 1);
+      assertEquals(expected, next);
+      size = next;
+      expected *= 2;
+    }
+  }
+}
+
+// End UnsynchronizedBufferTest.java


[3/7] calcite git commit: [CALCITE-1093] Reduce impact of ArrayList performance

Posted by el...@apache.org.
[CALCITE-1093] Reduce impact of ArrayList performance


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

Branch: refs/heads/master
Commit: ea41b3412f099c8f1051f1b324b1ba03d644962a
Parents: 1a9be67
Author: Josh Elser <el...@apache.org>
Authored: Wed Mar 2 17:43:33 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Wed Mar 2 19:06:33 2016 -0500

----------------------------------------------------------------------
 .../org/apache/calcite/avatica/jdbc/JdbcMeta.java   | 16 ++++++++--------
 .../apache/calcite/avatica/remote/LocalService.java |  2 +-
 2 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/ea41b341/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 becb19d..dfe7f99 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
@@ -788,7 +788,7 @@ public class JdbcMeta implements Meta {
       if (null == statementInfo) {
         throw new NoSuchStatementException(h);
       }
-      final List<MetaResultSet> resultSets = new ArrayList<>();
+      final List<MetaResultSet> resultSets;
       final PreparedStatement preparedStatement =
           (PreparedStatement) statementInfo.statement;
 
@@ -816,16 +816,16 @@ public class JdbcMeta implements Meta {
 
         if (statementInfo.getResultSet() == null) {
           frame = Frame.EMPTY;
-          resultSets.add(JdbcResultSet.empty(h.connectionId, h.id, signature2));
+          resultSets = Collections.<MetaResultSet>singletonList(
+              JdbcResultSet.empty(h.connectionId, h.id, signature2));
         } else {
-          resultSets.add(
-              JdbcResultSet.create(h.connectionId, h.id,
-                  statementInfo.getResultSet(), maxRowCount, signature2));
+          resultSets = Collections.<MetaResultSet>singletonList(
+              JdbcResultSet.create(h.connectionId, h.id, statementInfo.getResultSet(),
+                  maxRowCount, signature2));
         }
       } else {
-        resultSets.add(
-            JdbcResultSet.count(
-                h.connectionId, h.id, preparedStatement.getUpdateCount()));
+        resultSets = Collections.<MetaResultSet>singletonList(
+            JdbcResultSet.count(h.connectionId, h.id, preparedStatement.getUpdateCount()));
       }
 
       return new ExecuteResult(resultSets);

http://git-wip-us.apache.org/repos/asf/calcite/blob/ea41b341/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 e9c4367..11d15c9 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
@@ -233,7 +233,7 @@ public class LocalService implements Service {
       final Meta.ExecuteResult executeResult = meta.execute(request.statementHandle,
           request.parameterValues, request.maxRowCount);
 
-      final List<ResultSetResponse> results = new ArrayList<>();
+      final List<ResultSetResponse> results = new ArrayList<>(executeResult.resultSets.size());
       for (Meta.MetaResultSet metaResultSet : executeResult.resultSets) {
         results.add(toResponse(metaResultSet));
       }