You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2014/12/09 22:49:50 UTC

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

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
new file mode 100644
index 0000000..68f8782
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/JsonService.java
@@ -0,0 +1,100 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.io.StringWriter;
+
+/**
+ * Implementation of {@link org.apache.calcite.avatica.remote.Service}
+ * that encodes requests and responses as JSON.
+ */
+public abstract class JsonService implements Service {
+  protected final ObjectMapper mapper;
+  protected final StringWriter w = new StringWriter();
+
+  public JsonService() {
+    mapper = new ObjectMapper();
+    mapper.configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
+    mapper.configure(JsonParser.Feature.ALLOW_SINGLE_QUOTES, true);
+  }
+
+  /** Derived class should implement this method to transport requests and
+   * responses to and from the peer service. */
+  public abstract String apply(String request);
+
+  private <T> T decode(String response, Class<T> valueType) throws IOException {
+    return mapper.readValue(response, valueType);
+  }
+
+  private <T> String encode(T request) throws IOException {
+    assert w.getBuffer().length() == 0;
+    mapper.writeValue(w, request);
+    final String s = w.toString();
+    w.getBuffer().setLength(0);
+    return s;
+  }
+
+  protected RuntimeException handle(IOException e) {
+    return new RuntimeException(e);
+  }
+
+  public ResultSetResponse apply(CatalogsRequest request) {
+    try {
+      return decode(apply(encode(request)), ResultSetResponse.class);
+    } catch (IOException e) {
+      throw handle(e);
+    }
+  }
+
+  public ResultSetResponse apply(SchemasRequest request) {
+    try {
+      return decode(apply(encode(request)), ResultSetResponse.class);
+    } catch (IOException e) {
+      throw handle(e);
+    }
+  }
+
+  public PrepareResponse apply(PrepareRequest request) {
+    try {
+      return decode(apply(encode(request)), PrepareResponse.class);
+    } catch (IOException e) {
+      throw handle(e);
+    }
+  }
+
+  public ResultSetResponse apply(PrepareAndExecuteRequest request) {
+    try {
+      return decode(apply(encode(request)), ResultSetResponse.class);
+    } catch (IOException e) {
+      throw handle(e);
+    }
+  }
+
+  public CreateStatementResponse apply(CreateStatementRequest request) {
+    try {
+      return decode(apply(encode(request)), CreateStatementResponse.class);
+    } catch (IOException e) {
+      throw handle(e);
+    }
+  }
+}
+
+// End JsonService.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalJsonService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalJsonService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalJsonService.java
new file mode 100644
index 0000000..66b4b7d
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalJsonService.java
@@ -0,0 +1,47 @@
+/*
+ * 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 java.io.IOException;
+
+/**
+ * Implementation of {@link org.apache.calcite.avatica.remote.Service}
+ * that goes to an in-process instance of {@code Service}.
+ */
+public class LocalJsonService extends JsonService {
+  private final Service service;
+
+  public LocalJsonService(Service service) {
+    this.service = service;
+  }
+
+  @Override public String apply(String request) {
+    try {
+      Request request2 = mapper.readValue(request, Request.class);
+      Response response2 = request2.accept(service);
+      assert w.getBuffer().length() == 0;
+      mapper.writeValue(w, response2);
+      final String response = w.toString();
+      w.getBuffer().setLength(0);
+      return response;
+    } catch (IOException e) {
+      throw handle(e);
+    }
+  }
+}
+
+// End LocalJsonService.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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
new file mode 100644
index 0000000..5ee3d16
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
@@ -0,0 +1,129 @@
+/*
+ * 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.Meta;
+import org.apache.calcite.avatica.MetaImpl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Implementation of {@link Service} that talks to a local {@link Meta}.
+ */
+public class LocalService implements Service {
+  final Meta meta;
+
+  public LocalService(Meta meta) {
+    this.meta = meta;
+  }
+
+  private static <E> List<E> list(Iterable<E> iterable) {
+    if (iterable instanceof List) {
+      return (List<E>) iterable;
+    }
+    final List<E> rowList = new ArrayList<E>();
+    for (E row : iterable) {
+      rowList.add(row);
+    }
+    return rowList;
+  }
+
+  /** Converts a result set (not serializable) into a serializable response. */
+  public ResultSetResponse toResponse(Meta.MetaResultSet resultSet) {
+    Meta.CursorFactory cursorFactory = resultSet.signature.cursorFactory;
+    final List<Object> list;
+    if (resultSet.iterable != null) {
+      list = list(resultSet.iterable);
+      switch (cursorFactory.style) {
+      case ARRAY:
+        cursorFactory = Meta.CursorFactory.LIST;
+        break;
+      case MAP:
+        break;
+      default:
+        cursorFactory = Meta.CursorFactory.map(cursorFactory.fieldNames);
+      }
+    } else {
+      //noinspection unchecked
+      list = (List<Object>) (List) list2(resultSet);
+      cursorFactory = Meta.CursorFactory.LIST;
+    }
+    Meta.Signature signature = resultSet.signature;
+    if (cursorFactory != resultSet.signature.cursorFactory) {
+      signature = signature.setCursorFactory(cursorFactory);
+    }
+    return new ResultSetResponse(resultSet.statementId, resultSet.ownStatement,
+        signature, list);
+  }
+
+  private List<List<Object>> list2(Meta.MetaResultSet resultSet) {
+    List<List<Object>> list = new ArrayList<List<Object>>();
+    return MetaImpl.collect(resultSet.signature.cursorFactory,
+        meta.createIterable(new Meta.StatementHandle(resultSet.statementId),
+            resultSet.signature,
+            resultSet.iterable),
+        list);
+  }
+
+  public ResultSetResponse apply(CatalogsRequest request) {
+    final Meta.MetaResultSet resultSet = meta.getCatalogs();
+    return toResponse(resultSet);
+  }
+
+  public ResultSetResponse apply(SchemasRequest request) {
+    final Meta.MetaResultSet resultSet =
+        meta.getSchemas(request.catalog, Meta.Pat.of(request.schemaPattern));
+    return toResponse(resultSet);
+  }
+
+  public PrepareResponse apply(PrepareRequest request) {
+    final Meta.StatementHandle h =
+        new Meta.StatementHandle(request.statementId);
+    final Meta.Signature signature =
+        meta.prepare(h, request.sql, request.maxRowCount);
+    return new PrepareResponse(signature);
+  }
+
+  public ResultSetResponse apply(PrepareAndExecuteRequest request) {
+    final Meta.StatementHandle h =
+        new Meta.StatementHandle(request.statementId);
+    final Meta.MetaResultSet resultSet =
+        meta.prepareAndExecute(h, request.sql, request.maxRowCount,
+            new Meta.PrepareCallback() {
+              public Object getMonitor() {
+                return LocalService.class;
+              }
+
+              public void clear() {}
+
+              public void assign(Meta.Signature signature,
+                  Iterable<Object> iterable) {}
+
+              public void execute() {}
+            });
+    return toResponse(resultSet);
+  }
+
+  public CreateStatementResponse apply(CreateStatementRequest request) {
+    final Meta.StatementHandle h =
+        meta.createStatement(new Meta.ConnectionHandle(request.connectionId));
+    return new CreateStatementResponse(h.id);
+  }
+}
+
+// End LocalService.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/avatica/src/main/java/org/apache/calcite/avatica/remote/MockJsonService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/MockJsonService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/MockJsonService.java
new file mode 100644
index 0000000..7fe7ea8
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/MockJsonService.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.avatica.remote;
+
+import org.apache.calcite.avatica.AvaticaConnection;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Mock implementation of {@link Service}
+ * that encodes its requests and responses as JSON
+ * and looks up responses from a pre-defined map.
+ */
+public class MockJsonService extends JsonService {
+  private final Map<String, String> map;
+
+  public MockJsonService(Map<String, String> map) {
+    this.map = map;
+  }
+
+  @Override public String apply(String request) {
+    final String response = map.get(request);
+    if (response == null) {
+      throw new RuntimeException("No response for " + request);
+    }
+    return response;
+  }
+
+  /** Factory that creates a {@code MockJsonService}. */
+  public static class Factory implements Service.Factory {
+    public Service create(AvaticaConnection connection) {
+      final Map<String, String> map1 = new HashMap<String, String>();
+      map1.put(
+          "{\"request\":\"getSchemas\",\"catalog\":null,\"schemaPattern\":{\"s\":null}}",
+          "{\"response\":\"resultSet\", rows: []}");
+      return new MockJsonService(map1);
+    }
+  }
+}
+
+// End MockJsonService.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.java
new file mode 100644
index 0000000..89ef490
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteMeta.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.remote;
+
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.MetaImpl;
+
+import java.sql.SQLException;
+
+/**
+ * Implementation of {@link Meta} for the remote driver.
+ */
+class RemoteMeta extends MetaImpl {
+  final Service service;
+
+  public RemoteMeta(AvaticaConnection connection, Service service) {
+    super(connection);
+    this.service = service;
+  }
+
+  private MetaResultSet toResultSet(Service.ResultSetResponse response) {
+    final Signature signature0 = response.signature;
+    return new MetaResultSet(response.statementId, response.ownStatement,
+        signature0, response.rows);
+  }
+
+  @Override public StatementHandle createStatement(ConnectionHandle ch) {
+    final Service.CreateStatementResponse response =
+        service.apply(new Service.CreateStatementRequest(ch.id));
+    return new StatementHandle(response.id);
+  }
+
+  @Override public MetaResultSet getCatalogs() {
+    final Service.ResultSetResponse response =
+        service.apply(new Service.CatalogsRequest());
+    return toResultSet(response);
+  }
+
+  @Override public MetaResultSet getSchemas(String catalog, Pat schemaPattern) {
+    final Service.ResultSetResponse response =
+        service.apply(new Service.SchemasRequest(catalog, schemaPattern.s));
+    return toResultSet(response);
+  }
+
+  @Override public Signature prepare(StatementHandle h, String sql,
+      int maxRowCount) {
+    final Service.PrepareResponse response =
+        service.apply(new Service.PrepareRequest(h.id, sql, maxRowCount));
+    return response.signature;
+  }
+
+  @Override public MetaResultSet prepareAndExecute(StatementHandle h,
+      String sql, int maxRowCount, PrepareCallback callback) {
+    final Service.ResultSetResponse response;
+    try {
+      synchronized (callback.getMonitor()) {
+        callback.clear();
+        response = service.apply(
+            new Service.PrepareAndExecuteRequest(h.id, sql, maxRowCount));
+        callback.assign(response.signature, response.rows);
+      }
+      callback.execute();
+      return toResultSet(response);
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}
+
+// End RemoteMeta.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteService.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteService.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteService.java
new file mode 100644
index 0000000..7cbdcf0
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/RemoteService.java
@@ -0,0 +1,64 @@
+/*
+ * 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 java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+
+/**
+ * Implementation of {@link org.apache.calcite.avatica.remote.Service}
+ * that translates requests into JSON and sends them to a remote server,
+ * usually an HTTP server.
+ */
+public class RemoteService extends JsonService {
+  private final URL url;
+
+  public RemoteService(URL url) {
+    this.url = url;
+  }
+
+  @Override public String apply(String request) {
+    try {
+      final HttpURLConnection connection =
+          (HttpURLConnection) url.openConnection();
+      connection.setRequestMethod("POST");
+      connection.setRequestProperty("request", request);
+      final int responseCode = connection.getResponseCode();
+      if (responseCode != HttpURLConnection.HTTP_OK) {
+        throw new RuntimeException("response code " + responseCode);
+      }
+      final InputStream inputStream = connection.getInputStream();
+      final byte[] bytes = new byte[4096];
+      final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      for (;;) {
+        int count = inputStream.read(bytes, 0, bytes.length);
+        if (count < 0) {
+          break;
+        }
+        baos.write(bytes, 0, count);
+      }
+      return baos.toString();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}
+
+// End RemoteService.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/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
new file mode 100644
index 0000000..cad38dd
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/Service.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.avatica.remote;
+
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.Meta;
+
+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 java.util.List;
+
+/**
+ * API for request-response calls to an Avatica server.
+ */
+public interface Service {
+  ResultSetResponse apply(CatalogsRequest request);
+  ResultSetResponse apply(SchemasRequest request);
+  PrepareResponse apply(PrepareRequest request);
+  ResultSetResponse apply(PrepareAndExecuteRequest request);
+  CreateStatementResponse apply(CreateStatementRequest request);
+
+  /** Factory that creates a {@code Service}. */
+  interface Factory {
+    Service create(AvaticaConnection connection);
+  }
+
+  /** Base class for all service request messages. */
+  @JsonTypeInfo(
+      use = JsonTypeInfo.Id.NAME,
+      property = "request",
+      defaultImpl = SchemasRequest.class)
+  @JsonSubTypes({
+      @JsonSubTypes.Type(value = CatalogsRequest.class, name = "getCatalogs"),
+      @JsonSubTypes.Type(value = SchemasRequest.class, name = "getSchemas"),
+      @JsonSubTypes.Type(value = PrepareRequest.class, name = "prepare"),
+      @JsonSubTypes.Type(value = PrepareAndExecuteRequest.class,
+          name = "prepareAndExecute"),
+      @JsonSubTypes.Type(value = CreateStatementRequest.class,
+          name = "createStatement") })
+  abstract class Request {
+    abstract Response accept(Service service);
+  }
+
+  /** Base class for all service response messages. */
+  @JsonTypeInfo(
+      use = JsonTypeInfo.Id.NAME,
+      property = "response",
+      defaultImpl = ResultSetResponse.class)
+  @JsonSubTypes({
+      @JsonSubTypes.Type(value = ResultSetResponse.class, name = "resultSet"),
+      @JsonSubTypes.Type(value = PrepareResponse.class, name = "prepare"),
+      @JsonSubTypes.Type(value = CreateStatementResponse.class,
+          name = "createStatement") })
+  abstract class Response {
+  }
+
+  /** Request for
+   * {@link org.apache.calcite.avatica.Meta#getCatalogs()}. */
+  class CatalogsRequest extends Request {
+    ResultSetResponse accept(Service service) {
+      return service.apply(this);
+    }
+  }
+
+  /** Request for
+   * {@link Meta#getSchemas(String, org.apache.calcite.avatica.Meta.Pat)}. */
+  class SchemasRequest extends Request {
+    public final String catalog;
+    public final String schemaPattern;
+
+    @JsonCreator
+    public SchemasRequest(@JsonProperty("catalog") String catalog,
+        @JsonProperty("schemaPattern") String schemaPattern) {
+      this.catalog = catalog;
+      this.schemaPattern = schemaPattern;
+    }
+
+    ResultSetResponse accept(Service service) {
+      return service.apply(this);
+    }
+  }
+
+  /** Response that contains a result set.
+   *
+   * <p>Several types of request, including
+   * {@link org.apache.calcite.avatica.Meta#getCatalogs()} and
+   * {@link org.apache.calcite.avatica.Meta#getSchemas(String, org.apache.calcite.avatica.Meta.Pat)}
+   * return this response. */
+  class ResultSetResponse extends Response {
+    public final int statementId;
+    public final boolean ownStatement;
+    public final Meta.Signature signature;
+    public final List<Object> rows;
+
+    @JsonCreator
+    public ResultSetResponse(@JsonProperty("statementId") int statementId,
+        @JsonProperty("ownStatement") boolean ownStatement,
+        @JsonProperty("signature") Meta.Signature signature,
+        @JsonProperty("rows") List<Object> rows) {
+      this.statementId = statementId;
+      this.ownStatement = ownStatement;
+      this.signature = signature;
+      this.rows = rows;
+    }
+  }
+
+  /** Request for
+   * {@link org.apache.calcite.avatica.Meta#prepareAndExecute(org.apache.calcite.avatica.Meta.StatementHandle, String, int, org.apache.calcite.avatica.Meta.PrepareCallback)}. */
+  class PrepareAndExecuteRequest extends Request {
+    public final int statementId;
+    public final String sql;
+    public final int maxRowCount;
+
+    @JsonCreator
+    public PrepareAndExecuteRequest(
+        @JsonProperty("statementId") int statementId,
+        @JsonProperty("sql") String sql,
+        @JsonProperty("maxRowCount") int maxRowCount) {
+      this.statementId = statementId;
+      this.sql = sql;
+      this.maxRowCount = maxRowCount;
+    }
+
+    @Override ResultSetResponse accept(Service service) {
+      return service.apply(this);
+    }
+  }
+
+  /** Request for
+   * {@link org.apache.calcite.avatica.Meta#prepare(org.apache.calcite.avatica.Meta.StatementHandle, String, int)}. */
+  class PrepareRequest extends Request {
+    public final int statementId;
+    public final String sql;
+    public final int maxRowCount;
+
+    @JsonCreator
+    public PrepareRequest(@JsonProperty("statementId") int statementId,
+        @JsonProperty("sql") String sql,
+        @JsonProperty("maxRowCount") int maxRowCount) {
+      this.statementId = statementId;
+      this.sql = sql;
+      this.maxRowCount = maxRowCount;
+    }
+
+    @Override PrepareResponse accept(Service service) {
+      return service.apply(this);
+    }
+  }
+
+  /** Response from
+   * {@link org.apache.calcite.avatica.remote.Service.PrepareRequest}. */
+  class PrepareResponse extends Response {
+    public final Meta.Signature signature;
+
+    @JsonCreator
+    public PrepareResponse(
+        @JsonProperty("signature") Meta.Signature signature) {
+      this.signature = signature;
+    }
+  }
+
+  /** Request for
+   * {@link org.apache.calcite.avatica.Meta#createStatement(org.apache.calcite.avatica.Meta.ConnectionHandle)}. */
+  class CreateStatementRequest extends Request {
+    public final int connectionId;
+
+    @JsonCreator
+    public CreateStatementRequest(@JsonProperty("signature") int connectionId) {
+      this.connectionId = connectionId;
+    }
+
+    @Override CreateStatementResponse accept(Service service) {
+      return service.apply(this);
+    }
+  }
+
+  /** Response from
+   * {@link org.apache.calcite.avatica.remote.Service.CreateStatementRequest}. */
+  class CreateStatementResponse extends Response {
+    public final int id;
+
+    @JsonCreator
+    public CreateStatementResponse(@JsonProperty("id") int id) {
+      this.id = id;
+    }
+  }
+}
+
+// End Service.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/avatica/src/main/java/org/apache/calcite/avatica/remote/package-info.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/remote/package-info.java b/avatica/src/main/java/org/apache/calcite/avatica/remote/package-info.java
new file mode 100644
index 0000000..280dfb3
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/remote/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * JDBC driver that uses remote procedure calls.
+ */
+package org.apache.calcite.avatica.remote;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java b/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
new file mode 100644
index 0000000..301352f
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
@@ -0,0 +1,1105 @@
+/*
+ * 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.apache.calcite.avatica.AvaticaUtils;
+import org.apache.calcite.avatica.ColumnMetaData;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+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.SQLXML;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Base class for implementing a cursor.
+ *
+ * <p>Derived class needs to provide {@link Getter} and can override
+ * {@link org.apache.calcite.avatica.util.Cursor.Accessor} implementations if it
+ * wishes.</p>
+ */
+public abstract class AbstractCursor implements Cursor {
+  /**
+   * Slot into which each accessor should write whether the
+   * value returned was null.
+   */
+  protected final boolean[] wasNull = {false};
+
+  protected AbstractCursor() {
+  }
+
+  public boolean wasNull() {
+    return wasNull[0];
+  }
+
+  public List<Accessor> createAccessors(List<ColumnMetaData> types,
+      Calendar localCalendar, ArrayImpl.Factory factory) {
+    List<Accessor> accessors = new ArrayList<Accessor>();
+    for (ColumnMetaData type : types) {
+      accessors.add(
+          createAccessor(type, accessors.size(), localCalendar, factory));
+    }
+    return accessors;
+  }
+
+  protected Accessor createAccessor(ColumnMetaData type, int ordinal,
+      Calendar localCalendar, ArrayImpl.Factory factory) {
+    // Create an accessor appropriate to the underlying type; the accessor
+    // can convert to any type in the same family.
+    Getter getter = createGetter(ordinal);
+    switch (type.type.type) {
+    case Types.TINYINT:
+      return new ByteAccessor(getter);
+    case Types.SMALLINT:
+      return new ShortAccessor(getter);
+    case Types.INTEGER:
+      return new IntAccessor(getter);
+    case Types.BIGINT:
+      return new LongAccessor(getter);
+    case Types.BOOLEAN:
+      return new BooleanAccessor(getter);
+    case Types.FLOAT:
+    case Types.REAL:
+      return new FloatAccessor(getter);
+    case Types.DOUBLE:
+      return new DoubleAccessor(getter);
+    case Types.DECIMAL:
+      return new BigDecimalAccessor(getter);
+    case Types.CHAR:
+      switch (type.type.representation) {
+      case PRIMITIVE_CHAR:
+      case CHARACTER:
+        return new StringFromCharAccessor(getter, type.displaySize);
+      default:
+        return new FixedStringAccessor(getter, type.displaySize);
+      }
+    case Types.VARCHAR:
+      return new StringAccessor(getter);
+    case Types.BINARY:
+    case Types.VARBINARY:
+      return new BinaryAccessor(getter);
+    case Types.DATE:
+      switch (type.type.representation) {
+      case PRIMITIVE_INT:
+      case INTEGER:
+        return new DateFromIntAccessor(getter, localCalendar);
+      case JAVA_SQL_DATE:
+        return new DateAccessor(getter, localCalendar);
+      default:
+        throw new AssertionError("bad " + type.type.representation);
+      }
+    case Types.TIME:
+      switch (type.type.representation) {
+      case PRIMITIVE_INT:
+      case INTEGER:
+        return new TimeFromIntAccessor(getter, localCalendar);
+      case JAVA_SQL_TIME:
+        return new TimeAccessor(getter, localCalendar);
+      default:
+        throw new AssertionError("bad " + type.type.representation);
+      }
+    case Types.TIMESTAMP:
+      switch (type.type.representation) {
+      case PRIMITIVE_LONG:
+      case LONG:
+        return new TimestampFromLongAccessor(getter, localCalendar);
+      case JAVA_SQL_TIMESTAMP:
+        return new TimestampAccessor(getter, localCalendar);
+      case JAVA_UTIL_DATE:
+        return new TimestampFromUtilDateAccessor(getter, localCalendar);
+      default:
+        throw new AssertionError("bad " + type.type.representation);
+      }
+    case Types.ARRAY:
+      return new ArrayAccessor(getter,
+          ((ColumnMetaData.ArrayType) type.type).component, factory);
+    case Types.JAVA_OBJECT:
+    case Types.STRUCT:
+    case Types.OTHER: // e.g. map
+      if (type.type.typeName.startsWith("INTERVAL_")) {
+        int end = type.type.typeName.indexOf("(");
+        if (end < 0) {
+          end = type.type.typeName.length();
+        }
+        TimeUnitRange range =
+            TimeUnitRange.valueOf(
+                type.type.typeName.substring("INTERVAL_".length(), end));
+        if (range.monthly()) {
+          return new IntervalYearMonthAccessor(getter, range);
+        } else {
+          return new IntervalDayTimeAccessor(getter, range, type.scale);
+        }
+      }
+      return new ObjectAccessor(getter);
+    default:
+      throw new RuntimeException("unknown type " + type.type.type);
+    }
+  }
+
+  protected abstract Getter createGetter(int ordinal);
+
+  public abstract boolean next();
+
+  /** Accesses a timestamp value as a string.
+   * The timestamp is in SQL format (e.g. "2013-09-22 22:30:32"),
+   * not Java format ("2013-09-22 22:30:32.123"). */
+  private static String timestampAsString(long v, Calendar calendar) {
+    if (calendar != null) {
+      v -= calendar.getTimeZone().getOffset(v);
+    }
+    return DateTimeUtils.unixTimestampToString(v);
+  }
+
+  /** Accesses a date value as a string, e.g. "2013-09-22". */
+  private static String dateAsString(int v, Calendar calendar) {
+    AvaticaUtils.discard(calendar); // timezone shift doesn't make sense
+    return DateTimeUtils.unixDateToString(v);
+  }
+
+  /** Accesses a time value as a string, e.g. "22:30:32". */
+  private static String timeAsString(int v, Calendar calendar) {
+    if (calendar != null) {
+      v -= calendar.getTimeZone().getOffset(v);
+    }
+    return DateTimeUtils.unixTimeToString(v);
+  }
+
+  private static Date longToDate(long v, Calendar calendar) {
+    if (calendar != null) {
+      v -= calendar.getTimeZone().getOffset(v);
+    }
+    return new Date(v);
+  }
+
+  static Time intToTime(int v, Calendar calendar) {
+    if (calendar != null) {
+      v -= calendar.getTimeZone().getOffset(v);
+    }
+    return new Time(v);
+  }
+
+  static Timestamp longToTimestamp(long v, Calendar calendar) {
+    if (calendar != null) {
+      v -= calendar.getTimeZone().getOffset(v);
+    }
+    return new Timestamp(v);
+  }
+
+  /** Implementation of {@link Accessor}. */
+  static class AccessorImpl implements Accessor {
+    protected final Getter getter;
+
+    public AccessorImpl(Getter getter) {
+      assert getter != null;
+      this.getter = getter;
+    }
+
+    public boolean wasNull() {
+      return getter.wasNull();
+    }
+
+    public String getString() {
+      final Object o = getObject();
+      return o == null ? null : o.toString();
+    }
+
+    public boolean getBoolean() {
+      return getLong() != 0L;
+    }
+
+    public byte getByte() {
+      return (byte) getLong();
+    }
+
+    public short getShort() {
+      return (short) getLong();
+    }
+
+    public int getInt() {
+      return (int) getLong();
+    }
+
+    public long getLong() {
+      throw cannotConvert("long");
+    }
+
+    public float getFloat() {
+      return (float) getDouble();
+    }
+
+    public double getDouble() {
+      throw cannotConvert("double");
+    }
+
+    public BigDecimal getBigDecimal() {
+      throw cannotConvert("BigDecimal");
+    }
+
+    public BigDecimal getBigDecimal(int scale) {
+      throw cannotConvert("BigDecimal with scale");
+    }
+
+    public byte[] getBytes() {
+      throw cannotConvert("byte[]");
+    }
+
+    public InputStream getAsciiStream() {
+      throw cannotConvert("InputStream (ascii)");
+    }
+
+    public InputStream getUnicodeStream() {
+      throw cannotConvert("InputStream (unicode)");
+    }
+
+    public InputStream getBinaryStream() {
+      throw cannotConvert("InputStream (binary)");
+    }
+
+    public Object getObject() {
+      return getter.getObject();
+    }
+
+    public Reader getCharacterStream() {
+      throw cannotConvert("Reader");
+    }
+
+    private RuntimeException cannotConvert(String targetType) {
+      return new RuntimeException("cannot convert to " + targetType + " ("
+          + this + ")");
+    }
+
+    public Object getObject(Map<String, Class<?>> map) {
+      throw cannotConvert("Object (with map)");
+    }
+
+    public Ref getRef() {
+      throw cannotConvert("Ref");
+    }
+
+    public Blob getBlob() {
+      throw cannotConvert("Blob");
+    }
+
+    public Clob getClob() {
+      throw cannotConvert("Clob");
+    }
+
+    public Array getArray() {
+      throw cannotConvert("Array");
+    }
+
+    public Date getDate(Calendar calendar) {
+      throw cannotConvert("Date");
+    }
+
+    public Time getTime(Calendar calendar) {
+      throw cannotConvert("Time");
+    }
+
+    public Timestamp getTimestamp(Calendar calendar) {
+      throw cannotConvert("Timestamp");
+    }
+
+    public URL getURL() {
+      throw cannotConvert("URL");
+    }
+
+    public NClob getNClob() {
+      throw cannotConvert("NClob");
+    }
+
+    public SQLXML getSQLXML() {
+      throw cannotConvert("SQLXML");
+    }
+
+    public String getNString() {
+      throw cannotConvert("NString");
+    }
+
+    public Reader getNCharacterStream() {
+      throw cannotConvert("NCharacterStream");
+    }
+
+    public <T> T getObject(Class<T> type) {
+      throw cannotConvert("Object (with type)");
+    }
+  }
+
+  /**
+   * Accessor of exact numeric values. The subclass must implement the
+   * {@link #getLong()} method.
+   */
+  private abstract static class ExactNumericAccessor extends AccessorImpl {
+    public ExactNumericAccessor(Getter getter) {
+      super(getter);
+    }
+
+    public BigDecimal getBigDecimal(int scale) {
+      final long v = getLong();
+      if (v == 0 && getter.wasNull()) {
+        return null;
+      }
+      return BigDecimal.valueOf(v).setScale(scale, RoundingMode.DOWN);
+    }
+
+    public BigDecimal getBigDecimal() {
+      final long val = getLong();
+      if (val == 0 && getter.wasNull()) {
+        return null;
+      }
+      return BigDecimal.valueOf(val);
+    }
+
+    public double getDouble() {
+      return getLong();
+    }
+
+    public float getFloat() {
+      return getLong();
+    }
+
+    public abstract long getLong();
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a {@link Boolean};
+   * corresponds to {@link java.sql.Types#BOOLEAN}.
+   */
+  private static class BooleanAccessor extends ExactNumericAccessor {
+    public BooleanAccessor(Getter getter) {
+      super(getter);
+    }
+
+    public boolean getBoolean() {
+      Boolean o = (Boolean) getObject();
+      return o != null && o;
+    }
+
+    public long getLong() {
+      return getBoolean() ? 1 : 0;
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a {@link Byte};
+   * corresponds to {@link java.sql.Types#TINYINT}.
+   */
+  private static class ByteAccessor extends ExactNumericAccessor {
+    public ByteAccessor(Getter getter) {
+      super(getter);
+    }
+
+    public byte getByte() {
+      Byte o = (Byte) getObject();
+      return o == null ? 0 : o;
+    }
+
+    public long getLong() {
+      return getByte();
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a {@link Short};
+   * corresponds to {@link java.sql.Types#SMALLINT}.
+   */
+  private static class ShortAccessor extends ExactNumericAccessor {
+    public ShortAccessor(Getter getter) {
+      super(getter);
+    }
+
+    public short getShort() {
+      Short o = (Short) getObject();
+      return o == null ? 0 : o;
+    }
+
+    public long getLong() {
+      return getShort();
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is an {@link Integer};
+   * corresponds to {@link java.sql.Types#INTEGER}.
+   */
+  private static class IntAccessor extends ExactNumericAccessor {
+    public IntAccessor(Getter getter) {
+      super(getter);
+    }
+
+    public int getInt() {
+      Integer o = (Integer) super.getObject();
+      return o == null ? 0 : o;
+    }
+
+    public long getLong() {
+      return getInt();
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a {@link Long};
+   * corresponds to {@link java.sql.Types#BIGINT}.
+   */
+  private static class LongAccessor extends ExactNumericAccessor {
+    public LongAccessor(Getter getter) {
+      super(getter);
+    }
+
+    public long getLong() {
+      Long o = (Long) super.getObject();
+      return o == null ? 0 : o;
+    }
+  }
+
+  /**
+   * Accessor of values that are {@link Double} or null.
+   */
+  private abstract static class ApproximateNumericAccessor
+      extends AccessorImpl {
+    public ApproximateNumericAccessor(Getter getter) {
+      super(getter);
+    }
+
+    public BigDecimal getBigDecimal(int scale) {
+      final double v = getDouble();
+      if (v == 0d && getter.wasNull()) {
+        return null;
+      }
+      return BigDecimal.valueOf(v).setScale(scale, RoundingMode.DOWN);
+    }
+
+    public BigDecimal getBigDecimal() {
+      final double v = getDouble();
+      if (v == 0 && getter.wasNull()) {
+        return null;
+      }
+      return BigDecimal.valueOf(v);
+    }
+
+    public abstract double getDouble();
+
+    public long getLong() {
+      return (long) getDouble();
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a {@link Float};
+   * corresponds to {@link java.sql.Types#FLOAT}.
+   */
+  private static class FloatAccessor extends ApproximateNumericAccessor {
+    public FloatAccessor(Getter getter) {
+      super(getter);
+    }
+
+    public float getFloat() {
+      Float o = (Float) getObject();
+      return o == null ? 0f : o;
+    }
+
+    public double getDouble() {
+      return getFloat();
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a {@link Double};
+   * corresponds to {@link java.sql.Types#DOUBLE}.
+   */
+  private static class DoubleAccessor extends ApproximateNumericAccessor {
+    public DoubleAccessor(Getter getter) {
+      super(getter);
+    }
+
+    public double getDouble() {
+      Double o = (Double) getObject();
+      return o == null ? 0d : o;
+    }
+  }
+
+  /**
+   * Accessor of exact numeric values. The subclass must implement the
+   * {@link #getLong()} method.
+   */
+  private abstract static class BigNumberAccessor extends AccessorImpl {
+    public BigNumberAccessor(Getter getter) {
+      super(getter);
+    }
+
+    protected abstract Number getNumber();
+
+    public double getDouble() {
+      Number number = getNumber();
+      return number == null ? 0d : number.doubleValue();
+    }
+
+    public float getFloat() {
+      Number number = getNumber();
+      return number == null ? 0f : number.floatValue();
+    }
+
+    public long getLong() {
+      Number number = getNumber();
+      return number == null ? 0L : number.longValue();
+    }
+
+    public int getInt() {
+      Number number = getNumber();
+      return number == null ? 0 : number.intValue();
+    }
+
+    public short getShort() {
+      Number number = getNumber();
+      return number == null ? 0 : number.shortValue();
+    }
+
+    public byte getByte() {
+      Number number = getNumber();
+      return number == null ? 0 : number.byteValue();
+    }
+
+    public boolean getBoolean() {
+      Number number = getNumber();
+      return number != null && number.doubleValue() != 0;
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a {@link BigDecimal};
+   * corresponds to {@link java.sql.Types#DECIMAL}.
+   */
+  private static class BigDecimalAccessor extends BigNumberAccessor {
+    public BigDecimalAccessor(Getter getter) {
+      super(getter);
+    }
+
+    protected Number getNumber() {
+      return (Number) getObject();
+    }
+
+    public BigDecimal getBigDecimal(int scale) {
+      return (BigDecimal) getObject();
+    }
+
+    public BigDecimal getBigDecimal() {
+      return (BigDecimal) getObject();
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a {@link String};
+   * corresponds to {@link java.sql.Types#CHAR}
+   * and {@link java.sql.Types#VARCHAR}.
+   */
+  private static class StringAccessor extends AccessorImpl {
+    public StringAccessor(Getter getter) {
+      super(getter);
+    }
+
+    public String getString() {
+      return (String) getObject();
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a {@link String};
+   * corresponds to {@link java.sql.Types#CHAR}.
+   */
+  private static class FixedStringAccessor extends StringAccessor {
+    protected final Spacer spacer;
+
+    public FixedStringAccessor(Getter getter, int length) {
+      super(getter);
+      this.spacer = new Spacer(length);
+    }
+
+    public String getString() {
+      String s = super.getString();
+      if (s == null) {
+        return null;
+      }
+      return spacer.padRight(s);
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a {@link String};
+   * corresponds to {@link java.sql.Types#CHAR}.
+   */
+  private static class StringFromCharAccessor extends FixedStringAccessor {
+    public StringFromCharAccessor(Getter getter, int length) {
+      super(getter, length);
+    }
+
+    public String getString() {
+      Character s = (Character) super.getObject();
+      if (s == null) {
+        return null;
+      }
+      return spacer.padRight(s.toString());
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is an array of
+   * {@link org.apache.calcite.avatica.util.ByteString} values;
+   * corresponds to {@link java.sql.Types#BINARY}
+   * and {@link java.sql.Types#VARBINARY}.
+   */
+  private static class BinaryAccessor extends AccessorImpl {
+    public BinaryAccessor(Getter getter) {
+      super(getter);
+    }
+
+    public byte[] getBytes() {
+      final ByteString o = (ByteString) getObject();
+      return o == null ? null : o.getBytes();
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a DATE,
+   * in its default representation {@code int};
+   * corresponds to {@link java.sql.Types#DATE}.
+   */
+  private static class DateFromIntAccessor extends IntAccessor {
+    private final Calendar localCalendar;
+
+    public DateFromIntAccessor(Getter getter, Calendar localCalendar) {
+      super(getter);
+      this.localCalendar = localCalendar;
+    }
+
+    @Override public Object getObject() {
+      return getDate(localCalendar);
+    }
+
+    @Override public Date getDate(Calendar calendar) {
+      final int v = getInt();
+      if (v == 0 && getter.wasNull()) {
+        return null;
+      }
+      return longToDate((long) v * DateTimeUtils.MILLIS_PER_DAY, calendar);
+    }
+
+    @Override public Timestamp getTimestamp(Calendar calendar) {
+      final int v = getInt();
+      if (v == 0 && getter.wasNull()) {
+        return null;
+      }
+      return longToTimestamp((long) v * DateTimeUtils.MILLIS_PER_DAY, calendar);
+    }
+
+    @Override public String getString() {
+      final int v = getInt();
+      if (v == 0 && wasNull()) {
+        return null;
+      }
+      return dateAsString(v, null);
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a Time,
+   * in its default representation {@code int};
+   * corresponds to {@link java.sql.Types#TIME}.
+   */
+  private static class TimeFromIntAccessor extends IntAccessor {
+    private final Calendar localCalendar;
+
+    public TimeFromIntAccessor(Getter getter, Calendar localCalendar) {
+      super(getter);
+      this.localCalendar = localCalendar;
+    }
+
+    @Override public Object getObject() {
+      return getTime(localCalendar);
+    }
+
+    @Override public Time getTime(Calendar calendar) {
+      final int v = getInt();
+      if (v == 0 && wasNull()) {
+        return null;
+      }
+      return intToTime(v, calendar);
+    }
+
+    @Override public Timestamp getTimestamp(Calendar calendar) {
+      final long v = getLong();
+      if (v == 0 && wasNull()) {
+        return null;
+      }
+      return longToTimestamp(v, calendar);
+    }
+
+    @Override public String getString() {
+      final int v = getInt();
+      if (v == 0 && wasNull()) {
+        return null;
+      }
+      return timeAsString(v, null);
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a TIMESTAMP,
+   * in its default representation {@code long};
+   * corresponds to {@link java.sql.Types#TIMESTAMP}.
+   */
+  private static class TimestampFromLongAccessor extends LongAccessor {
+    private final Calendar localCalendar;
+
+    public TimestampFromLongAccessor(Getter getter, Calendar localCalendar) {
+      super(getter);
+      this.localCalendar = localCalendar;
+    }
+
+    @Override public Object getObject() {
+      return getTimestamp(localCalendar);
+    }
+
+    @Override public Timestamp getTimestamp(Calendar calendar) {
+      final long v = getLong();
+      if (v == 0 && wasNull()) {
+        return null;
+      }
+      return longToTimestamp(v, calendar);
+    }
+
+    @Override public String getString() {
+      final long v = getLong();
+      if (v == 0L && wasNull()) {
+        return null;
+      }
+      return timestampAsString(v, null);
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a DATE,
+   * represented as a java.sql.Date;
+   * corresponds to {@link java.sql.Types#DATE}.
+   */
+  private static class DateAccessor extends ObjectAccessor {
+    private final Calendar localCalendar;
+
+    public DateAccessor(Getter getter, Calendar localCalendar) {
+      super(getter);
+      this.localCalendar = localCalendar;
+    }
+
+    @Override public Date getDate(Calendar calendar) {
+      java.sql.Date date = (Date) getObject();
+      if (date == null) {
+        return null;
+      }
+      if (calendar != null) {
+        long v = date.getTime();
+        v -= calendar.getTimeZone().getOffset(v);
+        date = new Date(v);
+      }
+      return date;
+    }
+
+    @Override public String getString() {
+      final int v = getInt();
+      if (v == 0 && wasNull()) {
+        return null;
+      }
+      return dateAsString(v, null);
+    }
+
+    @Override public long getLong() {
+      Date date = getDate(null);
+      return date == null
+          ? 0L
+          : (date.getTime() / DateTimeUtils.MILLIS_PER_DAY);
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a TIME,
+   * represented as a java.sql.Time;
+   * corresponds to {@link java.sql.Types#TIME}.
+   */
+  private static class TimeAccessor extends ObjectAccessor {
+    private final Calendar localCalendar;
+
+    public TimeAccessor(Getter getter, Calendar localCalendar) {
+      super(getter);
+      this.localCalendar = localCalendar;
+    }
+
+    @Override public Time getTime(Calendar calendar) {
+      Time date  = (Time) getObject();
+      if (date == null) {
+        return null;
+      }
+      if (calendar != null) {
+        long v = date.getTime();
+        v -= calendar.getTimeZone().getOffset(v);
+        date = new Time(v);
+      }
+      return date;
+    }
+
+    @Override public String getString() {
+      final int v = getInt();
+      if (v == 0 && wasNull()) {
+        return null;
+      }
+      return timeAsString(v, null);
+    }
+
+    @Override public long getLong() {
+      Time time = getTime(null);
+      return time == null ? 0L
+          : (time.getTime() % DateTimeUtils.MILLIS_PER_DAY);
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a TIMESTAMP,
+   * represented as a java.sql.Timestamp;
+   * corresponds to {@link java.sql.Types#TIMESTAMP}.
+   */
+  private static class TimestampAccessor extends ObjectAccessor {
+    private final Calendar localCalendar;
+
+    public TimestampAccessor(Getter getter, Calendar localCalendar) {
+      super(getter);
+      this.localCalendar = localCalendar;
+    }
+
+    @Override public Timestamp getTimestamp(Calendar calendar) {
+      Timestamp date  = (Timestamp) getObject();
+      if (date == null) {
+        return null;
+      }
+      if (calendar != null) {
+        long v = date.getTime();
+        v -= calendar.getTimeZone().getOffset(v);
+        date = new Timestamp(v);
+      }
+      return date;
+    }
+
+    @Override public String getString() {
+      final long v = getLong();
+      if (v == 0 && wasNull()) {
+        return null;
+      }
+      return timestampAsString(v, null);
+    }
+
+    @Override public long getLong() {
+      Timestamp timestamp = getTimestamp(null);
+      return timestamp == null ? 0 : timestamp.getTime();
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a TIMESTAMP,
+   * represented as a java.util.Date;
+   * corresponds to {@link java.sql.Types#TIMESTAMP}.
+   */
+  private static class TimestampFromUtilDateAccessor extends ObjectAccessor {
+    private final Calendar localCalendar;
+
+    public TimestampFromUtilDateAccessor(Getter getter,
+        Calendar localCalendar) {
+      super(getter);
+      this.localCalendar = localCalendar;
+    }
+
+    @Override public Timestamp getTimestamp(Calendar calendar) {
+      java.util.Date date  = (java.util.Date) getObject();
+      if (date == null) {
+        return null;
+      }
+      long v = date.getTime();
+      if (calendar != null) {
+        v -= calendar.getTimeZone().getOffset(v);
+      }
+      return new Timestamp(v);
+    }
+
+    @Override public String getString() {
+      java.util.Date date  = (java.util.Date) getObject();
+      if (date == null) {
+        return null;
+      }
+      return timestampAsString(date.getTime(), null);
+    }
+
+    @Override public long getLong() {
+      Timestamp timestamp = getTimestamp(localCalendar);
+      return timestamp == null ? 0 : timestamp.getTime();
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a {@code int};
+   * corresponds to {@link java.sql.Types#OTHER}.
+   */
+  private static class IntervalYearMonthAccessor extends IntAccessor {
+    private final TimeUnitRange range;
+
+    public IntervalYearMonthAccessor(Getter getter, TimeUnitRange range) {
+      super(getter);
+      this.range = range;
+    }
+
+    @Override public String getString() {
+      final int v = getInt();
+      if (v == 0 && wasNull()) {
+        return null;
+      }
+      return DateTimeUtils.intervalYearMonthToString(v, range);
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is a {@code long};
+   * corresponds to {@link java.sql.Types#OTHER}.
+   */
+  private static class IntervalDayTimeAccessor extends LongAccessor {
+    private final TimeUnitRange range;
+    private final int scale;
+
+    public IntervalDayTimeAccessor(Getter getter, TimeUnitRange range,
+        int scale) {
+      super(getter);
+      this.range = range;
+      this.scale = scale;
+    }
+
+    @Override public String getString() {
+      final long v = getLong();
+      if (v == 0 && wasNull()) {
+        return null;
+      }
+      return DateTimeUtils.intervalDayTimeToString(v, range, scale);
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is an ARRAY;
+   * corresponds to {@link java.sql.Types#ARRAY}.
+   */
+  private static class ArrayAccessor extends AccessorImpl {
+    private final ColumnMetaData.AvaticaType componentType;
+    private final ArrayImpl.Factory factory;
+
+    public ArrayAccessor(Getter getter,
+        ColumnMetaData.AvaticaType componentType, ArrayImpl.Factory factory) {
+      super(getter);
+      this.componentType = componentType;
+      this.factory = factory;
+    }
+
+    @Override public Object getObject() {
+      final Object object = super.getObject();
+      if (object == null || object instanceof List) {
+        return object;
+      }
+      // The object can be java array in case of user-provided class for row
+      // storage.
+      return AvaticaUtils.primitiveList(object);
+    }
+
+    @Override public Array getArray() {
+      final List list = (List) getObject();
+      if (list == null) {
+        return null;
+      }
+      return new ArrayImpl(list, componentType, factory);
+    }
+
+    @Override public String getString() {
+      final List o = (List) getObject();
+      if (o == null) {
+        return null;
+      }
+      final Iterator iterator = o.iterator();
+      if (!iterator.hasNext()) {
+        return "[]";
+      }
+      final StringBuilder buf = new StringBuilder("[");
+      for (;;) {
+        append(buf, iterator.next());
+        if (!iterator.hasNext()) {
+          return buf.append("]").toString();
+        }
+        buf.append(", ");
+      }
+    }
+
+    private void append(StringBuilder buf, Object o) {
+      if (o == null) {
+        buf.append("null");
+      } else if (o.getClass().isArray()) {
+        append(buf, AvaticaUtils.primitiveList(o));
+      } else {
+        buf.append(o);
+      }
+    }
+  }
+
+  /**
+   * Accessor that assumes that the underlying value is an OBJECT;
+   * corresponds to {@link java.sql.Types#JAVA_OBJECT}.
+   */
+  private static class ObjectAccessor extends AccessorImpl {
+    public ObjectAccessor(Getter getter) {
+      super(getter);
+    }
+  }
+
+  /** Gets a value from a particular field of the current record of this
+   * cursor. */
+  protected interface Getter {
+    Object getObject();
+
+    boolean wasNull();
+  }
+
+  /** Abstract implementation of {@link Getter}. */
+  protected abstract class AbstractGetter implements Getter {
+    public boolean wasNull() {
+      return wasNull[0];
+    }
+  }
+}
+
+// End AbstractCursor.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/avatica/src/main/java/org/apache/calcite/avatica/util/ArrayImpl.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/util/ArrayImpl.java b/avatica/src/main/java/org/apache/calcite/avatica/util/ArrayImpl.java
new file mode 100644
index 0000000..e953e6a
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/util/ArrayImpl.java
@@ -0,0 +1,175 @@
+/*
+ * 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.apache.calcite.avatica.ColumnMetaData;
+
+import java.sql.Array;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.List;
+import java.util.Map;
+
+/** Implementation of JDBC {@link Array}. */
+public class ArrayImpl implements Array {
+  private final ColumnMetaData.AvaticaType elementType;
+  private final Factory factory;
+  private final List list;
+
+  public ArrayImpl(List list, ColumnMetaData.AvaticaType elementType,
+      Factory factory) {
+    this.list = list;
+    this.elementType = elementType;
+    this.factory = factory;
+  }
+
+  public String getBaseTypeName() throws SQLException {
+    return elementType.typeName;
+  }
+
+  public int getBaseType() throws SQLException {
+    return elementType.type;
+  }
+
+  public Object getArray() throws SQLException {
+    return getArray(list);
+  }
+
+  /**
+   * Converts a list into an array.
+   *
+   * <p>If the elements of the list are primitives, converts to an array of
+   * primitives (e.g. {@code boolean[]}.</p>
+   *
+   * @param list List of objects
+   *
+   * @return array
+   * @throws ClassCastException   if any element is not of the box type
+   * @throws NullPointerException if any element is null
+   */
+  @SuppressWarnings("unchecked")
+  protected Object getArray(List list) throws SQLException {
+    int i = 0;
+    switch (elementType.representation) {
+    case PRIMITIVE_DOUBLE:
+      final double[] doubles = new double[list.size()];
+      for (double v : (List<Double>) list) {
+        doubles[i++] = v;
+      }
+      return doubles;
+    case PRIMITIVE_FLOAT:
+      final float[] floats = new float[list.size()];
+      for (float v : (List<Float>) list) {
+        floats[i++] = v;
+      }
+      return floats;
+    case PRIMITIVE_INT:
+      final int[] ints = new int[list.size()];
+      for (int v : (List<Integer>) list) {
+        ints[i++] = v;
+      }
+      return ints;
+    case PRIMITIVE_LONG:
+      final long[] longs = new long[list.size()];
+      for (long v : (List<Long>) list) {
+        longs[i++] = v;
+      }
+      return longs;
+    case PRIMITIVE_SHORT:
+      final short[] shorts = new short[list.size()];
+      for (short v : (List<Short>) list) {
+        shorts[i++] = v;
+      }
+      return shorts;
+    case PRIMITIVE_BOOLEAN:
+      final boolean[] booleans = new boolean[list.size()];
+      for (boolean v : (List<Boolean>) list) {
+        booleans[i++] = v;
+      }
+      return booleans;
+    case PRIMITIVE_BYTE:
+      final byte[] bytes = new byte[list.size()];
+      for (byte v : (List<Byte>) list) {
+        bytes[i++] = v;
+      }
+      return bytes;
+    case PRIMITIVE_CHAR:
+      final char[] chars = new char[list.size()];
+      for (char v : (List<Character>) list) {
+        chars[i++] = v;
+      }
+      return chars;
+    default:
+      // fall through
+    }
+    final Object[] objects = list.toArray();
+    switch (elementType.type) {
+    case Types.ARRAY:
+      final ColumnMetaData.ArrayType arrayType =
+          (ColumnMetaData.ArrayType) elementType;
+      for (i = 0; i < objects.length; i++) {
+        objects[i] =
+            new ArrayImpl((List) objects[i], arrayType.component, factory);
+      }
+    }
+    return objects;
+  }
+
+  public Object getArray(Map<String, Class<?>> map) throws SQLException {
+    throw new UnsupportedOperationException(); // TODO
+  }
+
+  public Object getArray(long index, int count) throws SQLException {
+    return getArray(list.subList((int) index, count));
+  }
+
+  public Object getArray(long index, int count, Map<String, Class<?>> map)
+      throws SQLException {
+    throw new UnsupportedOperationException(); // TODO
+  }
+
+  public ResultSet getResultSet() throws SQLException {
+    return factory.create(elementType, list);
+  }
+
+  public ResultSet getResultSet(Map<String, Class<?>> map)
+      throws SQLException {
+    throw new UnsupportedOperationException(); // TODO
+  }
+
+  public ResultSet getResultSet(long index, int count) throws SQLException {
+    throw new UnsupportedOperationException(); // TODO
+  }
+
+  public ResultSet getResultSet(long index, int count,
+      Map<String, Class<?>> map) throws SQLException {
+    throw new UnsupportedOperationException(); // TODO
+  }
+
+  public void free() throws SQLException {
+    // nothing to do
+  }
+
+  /** Factory that can create a result set based on a list of values. */
+  public interface Factory {
+    ResultSet create(ColumnMetaData.AvaticaType elementType,
+        Iterable<Object> iterable);
+  }
+}
+
+// End ArrayImpl.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/avatica/src/main/java/org/apache/calcite/avatica/util/ArrayIteratorCursor.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/util/ArrayIteratorCursor.java b/avatica/src/main/java/org/apache/calcite/avatica/util/ArrayIteratorCursor.java
new file mode 100644
index 0000000..0477984
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/util/ArrayIteratorCursor.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 java.util.Iterator;
+
+/**
+ * Implementation of {@link Cursor} on top of an
+ * {@link java.util.Iterator} that
+ * returns an array of {@link Object} for each row.
+ */
+public class ArrayIteratorCursor extends IteratorCursor<Object[]> {
+  /**
+   * Creates an ArrayEnumeratorCursor.
+   *
+   * @param iterator Iterator
+   */
+  public ArrayIteratorCursor(Iterator<Object[]> iterator) {
+    super(iterator);
+  }
+
+  protected Getter createGetter(int ordinal) {
+    return new ArrayGetter(ordinal);
+  }
+}
+
+// End ArrayIteratorCursor.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/avatica/src/main/java/org/apache/calcite/avatica/util/ByteString.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/util/ByteString.java b/avatica/src/main/java/org/apache/calcite/avatica/util/ByteString.java
new file mode 100644
index 0000000..26228f1
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/util/ByteString.java
@@ -0,0 +1,232 @@
+/*
+ * 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.Serializable;
+import java.util.Arrays;
+
+/**
+ * Collection of bytes.
+ *
+ * <p>ByteString is to bytes what {@link String} is to chars: It is immutable,
+ * implements equality ({@link #hashCode} and {@link #equals}),
+ * comparison ({@link #compareTo}) and
+ * {@link Serializable serialization} correctly.</p>
+ */
+public class ByteString implements Comparable<ByteString>, Serializable {
+  private final byte[] bytes;
+
+  /** An empty byte string. */
+  public static final ByteString EMPTY = new ByteString(new byte[0], false);
+
+  private static final char[] DIGITS = {
+    '0', '1', '2', '3', '4', '5', '6', '7',
+    '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'
+  };
+
+  /**
+   * Creates a ByteString.
+   *
+   * @param bytes Bytes
+   */
+  public ByteString(byte[] bytes) {
+    this(bytes.clone(), false);
+  }
+
+  // private constructor that does not copy
+  private ByteString(byte[] bytes, boolean dummy) {
+    this.bytes = bytes;
+  }
+
+  @Override public int hashCode() {
+    return Arrays.hashCode(bytes);
+  }
+
+  @Override public boolean equals(Object obj) {
+    return this == obj
+        || obj instanceof ByteString
+        && Arrays.equals(bytes, ((ByteString) obj).bytes);
+  }
+
+  public int compareTo(ByteString that) {
+    final byte[] v1 = bytes;
+    final byte[] v2 = that.bytes;
+    final int n = Math.min(v1.length, v2.length);
+    for (int i = 0; i < n; i++) {
+      byte c1 = v1[i];
+      byte c2 = v2[i];
+      if (c1 != c2) {
+        return c1 - c2;
+      }
+    }
+    return v1.length - v2.length;
+  }
+
+  /**
+   * Returns this byte string in hexadecimal format.
+   *
+   * @return Hexadecimal string
+   */
+  @Override public String toString() {
+    return toString(16);
+  }
+
+  /**
+   * Returns this byte string in a given base.
+   *
+   * @return String in given base
+   */
+  public String toString(int base) {
+    return toString(bytes, base);
+  }
+
+  /**
+   * Returns the given byte array in hexadecimal format.
+   *
+   * <p>For example, <tt>toString(new byte[] {0xDE, 0xAD})</tt>
+   * returns {@code "DEAD"}.</p>
+   *
+   * @param bytes Array of bytes
+   * @param base Base (2 or 16)
+   * @return String
+   */
+  public static String toString(byte[] bytes, int base) {
+    char[] chars;
+    int j = 0;
+    switch (base) {
+    case 2:
+      chars = new char[bytes.length * 8];
+      for (byte b : bytes) {
+        chars[j++] = DIGITS[(b & 0x80) >> 7];
+        chars[j++] = DIGITS[(b & 0x40) >> 6];
+        chars[j++] = DIGITS[(b & 0x20) >> 5];
+        chars[j++] = DIGITS[(b & 0x10) >> 4];
+        chars[j++] = DIGITS[(b & 0x08) >> 3];
+        chars[j++] = DIGITS[(b & 0x04) >> 2];
+        chars[j++] = DIGITS[(b & 0x02) >> 1];
+        chars[j++] = DIGITS[b & 0x01];
+      }
+      break;
+    case 16:
+      chars = new char[bytes.length * 2];
+      for (byte b : bytes) {
+        chars[j++] = DIGITS[(b & 0xF0) >> 4];
+        chars[j++] = DIGITS[b & 0x0F];
+      }
+      break;
+    default:
+      throw new IllegalArgumentException("bad base " + base);
+    }
+    return new String(chars, 0, j);
+  }
+
+  @SuppressWarnings({
+      "CloneDoesntCallSuperClone",
+      "CloneDoesntDeclareCloneNotSupportedException"
+  })
+  @Override public Object clone() {
+    return this;
+  }
+
+  /**
+   * Returns the number of bytes in this byte string.
+   *
+   * @return Length of this byte string
+   */
+  public int length() {
+    return bytes.length;
+  }
+
+  /**
+   * Returns the byte at a given position in the byte string.
+   *
+   * @param i Index
+   * @throws  IndexOutOfBoundsException
+   *          if the <tt>index</tt> argument is negative or not less than
+   *          <tt>length()</tt>
+   * @return Byte at given position
+   */
+  public byte byteAt(int i) {
+    return bytes[i];
+  }
+
+  /**
+   * Returns a ByteString that consists of a given range.
+   *
+   * @param start Start of range
+   * @param end Position after end of range
+   * @return Substring
+   */
+  public ByteString substring(int start, int end) {
+    byte[] bytes = Arrays.copyOfRange(this.bytes, start, end);
+    return new ByteString(bytes, false);
+  }
+
+  /**
+   * Returns a ByteString that starts at a given position.
+   *
+   * @param start Start of range
+   * @return Substring
+   */
+  public ByteString substring(int start) {
+    return substring(start, length());
+  }
+
+  /**
+   * Returns a copy of the byte array.
+   */
+  public byte[] getBytes() {
+    return bytes.clone();
+  }
+
+  /**
+   * Returns a ByteString consisting of the concatenation of this and another
+   * string.
+   *
+   * @param other Byte string to concatenate
+   * @return Combined byte string
+   */
+  public ByteString concat(ByteString other) {
+    int otherLen = other.length();
+    if (otherLen == 0) {
+      return this;
+    }
+    int len = bytes.length;
+    byte[] buf = Arrays.copyOf(bytes, len + otherLen);
+    System.arraycopy(other.bytes, 0, buf, len, other.bytes.length);
+    return new ByteString(buf, false);
+  }
+
+  /** Returns the position at which {@code seek} first occurs in this byte
+   * string, or -1 if it does not occur. */
+  public int indexOf(ByteString seek) {
+  iLoop:
+    for (int i = 0; i < bytes.length - seek.bytes.length + 1; i++) {
+      for (int j = 0;; j++) {
+        if (j == seek.bytes.length) {
+          return i;
+        }
+        if (bytes[i + j] != seek.bytes[j]) {
+          continue iLoop;
+        }
+      }
+    }
+    return -1;
+  }
+}
+
+// End ByteString.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/avatica/src/main/java/org/apache/calcite/avatica/util/Casing.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/util/Casing.java b/avatica/src/main/java/org/apache/calcite/avatica/util/Casing.java
new file mode 100644
index 0000000..5f13214
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/util/Casing.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+/** Policy for converting case of identifiers before storing them.
+ *
+ * <p>A database often has policies for quoted versus unquoted identifiers.
+ * For example, Oracle converts unquoted identifiers to upper-case, but
+ * quoted identifiers are unchanged.</p> */
+public enum Casing {
+  /** The case of identifiers is not changed. */
+  UNCHANGED,
+
+  /** Identifiers are converted to upper-case. */
+  TO_UPPER,
+
+  /** Identifiers are converted to lower-case. */
+  TO_LOWER
+}
+
+// End Casing.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/94752a60/avatica/src/main/java/org/apache/calcite/avatica/util/Cursor.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/org/apache/calcite/avatica/util/Cursor.java b/avatica/src/main/java/org/apache/calcite/avatica/util/Cursor.java
new file mode 100644
index 0000000..98270e7
--- /dev/null
+++ b/avatica/src/main/java/org/apache/calcite/avatica/util/Cursor.java
@@ -0,0 +1,146 @@
+/*
+ * 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.apache.calcite.avatica.ColumnMetaData;
+
+import java.io.Closeable;
+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.SQLException;
+import java.sql.SQLXML;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Calendar;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Interface to an iteration that is similar to, and can easily support,
+ * a JDBC {@link java.sql.ResultSet}, but is simpler to implement.
+ */
+public interface Cursor extends Closeable {
+  /**
+   * Creates a list of accessors, one per column.
+   *
+   *
+   * @param types List of column types, per {@link java.sql.Types}.
+   * @param localCalendar Calendar in local timezone
+   * @param factory Factory that creates sub-ResultSets when needed
+   * @return List of column accessors
+   */
+  List<Accessor> createAccessors(List<ColumnMetaData> types,
+      Calendar localCalendar, ArrayImpl.Factory factory);
+
+  /**
+   * Moves to the next row.
+   *
+   * @return Whether moved
+   *
+   * @throws SQLException on database error
+   */
+  boolean next() throws SQLException;
+
+  /**
+   * Closes this cursor and releases resources.
+   */
+  void close();
+
+  /**
+   * Returns whether the last value returned was null.
+   *
+   * @throws SQLException on database error
+   */
+  boolean wasNull() throws SQLException;
+
+  /**
+   * Accessor of a column value.
+   */
+  public interface Accessor {
+    boolean wasNull() throws SQLException;
+
+    String getString() throws SQLException;
+
+    boolean getBoolean() throws SQLException;
+
+    byte getByte() throws SQLException;
+
+    short getShort() throws SQLException;
+
+    int getInt() throws SQLException;
+
+    long getLong() throws SQLException;
+
+    float getFloat() throws SQLException;
+
+    double getDouble() throws SQLException;
+
+    BigDecimal getBigDecimal() throws SQLException;
+
+    BigDecimal getBigDecimal(int scale) throws SQLException;
+
+    byte[] getBytes() throws SQLException;
+
+    InputStream getAsciiStream() throws SQLException;
+
+    InputStream getUnicodeStream() throws SQLException;
+
+    InputStream getBinaryStream() throws SQLException;
+
+    Object getObject() throws SQLException;
+
+    Reader getCharacterStream() throws SQLException;
+
+    Object getObject(Map<String, Class<?>> map) throws SQLException;
+
+    Ref getRef() throws SQLException;
+
+    Blob getBlob() throws SQLException;
+
+    Clob getClob() throws SQLException;
+
+    Array getArray() throws SQLException;
+
+    Date getDate(Calendar calendar) throws SQLException;
+
+    Time getTime(Calendar calendar) throws SQLException;
+
+    Timestamp getTimestamp(Calendar calendar) throws SQLException;
+
+    URL getURL() throws SQLException;
+
+    NClob getNClob() throws SQLException;
+
+    SQLXML getSQLXML() throws SQLException;
+
+    String getNString() throws SQLException;
+
+    Reader getNCharacterStream() throws SQLException;
+
+    <T> T getObject(Class<T> type) throws SQLException;
+  }
+}
+
+// End Cursor.java