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 2017/04/24 22:12:55 UTC

[1/4] calcite-avatica git commit: [CALCITE-1050] Array support for Avatica

Repository: calcite-avatica
Updated Branches:
  refs/heads/master 01ec8834e -> 557f15cc7


http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/server/src/test/java/org/apache/calcite/avatica/remote/AvaticaServersForTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/avatica/remote/AvaticaServersForTest.java b/server/src/test/java/org/apache/calcite/avatica/remote/AvaticaServersForTest.java
new file mode 100644
index 0000000..8e5de3a
--- /dev/null
+++ b/server/src/test/java/org/apache/calcite/avatica/remote/AvaticaServersForTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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.ConnectionSpec;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.jdbc.JdbcMeta;
+import org.apache.calcite.avatica.remote.Driver.Serialization;
+import org.apache.calcite.avatica.server.AvaticaJsonHandler;
+import org.apache.calcite.avatica.server.AvaticaProtobufHandler;
+import org.apache.calcite.avatica.server.HttpServer;
+import org.apache.calcite.avatica.server.Main;
+import org.apache.calcite.avatica.server.Main.HandlerFactory;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * Utility class which encapsulates the setup required to write Avatica tests that run against
+ * servers using each serialization approach.
+ */
+public class AvaticaServersForTest {
+  private static final ConnectionSpec CONNECTION_SPEC = ConnectionSpec.HSQLDB;
+  private static final String[] SERVER_ARGS = { FullyRemoteJdbcMetaFactory.class.getName() };
+
+  private final Map<Serialization, HttpServer> serversBySerialization;
+
+  public AvaticaServersForTest() {
+    serversBySerialization = new HashMap<>();
+  }
+
+  /**
+   * Starts an Avatica server for each serialization type.
+   */
+  public void startServers() throws Exception {
+    // Bind to '0' to pluck an ephemeral port instead of expecting a certain one to be free
+    final HttpServer jsonServer = Main.start(SERVER_ARGS, 0, new HandlerFactory() {
+      @Override public AvaticaJsonHandler createHandler(Service service) {
+        return new AvaticaJsonHandler(service);
+      }
+    });
+    serversBySerialization.put(Serialization.JSON, jsonServer);
+
+    final HttpServer protobufServer = Main.start(SERVER_ARGS, 0, new HandlerFactory() {
+      @Override public AvaticaProtobufHandler createHandler(Service service) {
+        return new AvaticaProtobufHandler(service);
+      }
+    });
+    serversBySerialization.put(Serialization.PROTOBUF, protobufServer);
+  }
+
+  /**
+   * Stops the servers currently running.
+   *
+   * @throws Exception If there is an error stopping a server
+   */
+  public void stopServers() throws Exception {
+    Iterator<Entry<Serialization, HttpServer>> servers =
+        serversBySerialization.entrySet().iterator();
+    while (servers.hasNext()) {
+      try {
+        servers.next().getValue().stop();
+      } finally {
+        // Still remove it if we failed to stop it
+        servers.remove();
+      }
+    }
+  }
+
+  /**
+   * Computes an array of parameters to support JUnit's parameterized tests. The Object array
+   * actually contains a {@link Serialization} and the {@link HttpServer} instance in that order.
+   *
+   * @return A list of arrays of Serialization and HttpServer pairs.
+   */
+  public List<Object[]> getJUnitParameters() {
+    List<Object[]> params = new ArrayList<>(serversBySerialization.size());
+
+    for (Entry<Serialization, HttpServer> servers : serversBySerialization.entrySet()) {
+      params.add(new Object[]{ servers.getKey(), servers.getValue() });
+    }
+
+    return params;
+  }
+
+  /**
+   * Computes the JDBC url for the Avatica server running on localhost, bound to the given port,
+   * and using the given serialization.
+   *
+   * @param port The port the Avatica server is listening on.
+   * @param serialization The serialization the Avatica server is using.
+   * @return A JDBC server to the local Avatica server.
+   */
+  public String getJdbcUrl(int port, Serialization serialization) {
+    return "jdbc:avatica:remote:url=http://localhost:" + port + ";serialization="
+        + serialization.name();
+  }
+
+  /** Factory that provides a {@link JdbcMeta}. */
+  public static class FullyRemoteJdbcMetaFactory implements Meta.Factory {
+
+    private static JdbcMeta instance = null;
+
+    static JdbcMeta getInstance() {
+      if (instance == null) {
+        try {
+          instance = new JdbcMeta(CONNECTION_SPEC.url, CONNECTION_SPEC.username,
+              CONNECTION_SPEC.password);
+        } catch (SQLException e) {
+          throw new RuntimeException(e);
+        }
+      }
+      return instance;
+    }
+
+    @Override public Meta create(List<String> args) {
+      return getInstance();
+    }
+  }
+}
+
+// End AvaticaServersForTest.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java b/server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
index ebd3c76..4cfcc1b 100644
--- a/server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
@@ -26,13 +26,10 @@ import org.apache.calcite.avatica.ConnectionSpec;
 import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.Meta.DatabaseProperty;
 import org.apache.calcite.avatica.jdbc.JdbcMeta;
+import org.apache.calcite.avatica.remote.AvaticaServersForTest.FullyRemoteJdbcMetaFactory;
 import org.apache.calcite.avatica.remote.Service.ErrorResponse;
 import org.apache.calcite.avatica.remote.Service.Response;
-import org.apache.calcite.avatica.server.AvaticaJsonHandler;
-import org.apache.calcite.avatica.server.AvaticaProtobufHandler;
 import org.apache.calcite.avatica.server.HttpServer;
-import org.apache.calcite.avatica.server.Main;
-import org.apache.calcite.avatica.server.Main.HandlerFactory;
 import org.apache.calcite.avatica.util.ArrayImpl;
 import org.apache.calcite.avatica.util.FilteredConstants;
 
@@ -62,7 +59,6 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
@@ -86,58 +82,30 @@ import static org.junit.Assert.fail;
 /** Tests covering {@link RemoteMeta}. */
 @RunWith(Parameterized.class)
 public class RemoteMetaTest {
+  private static final AvaticaServersForTest SERVERS = new AvaticaServersForTest();
   private static final Random RANDOM = new Random();
-  private static final ConnectionSpec CONNECTION_SPEC = ConnectionSpec.HSQLDB;
-
-  // Keep a reference to the servers we start to clean them up after
-  private static final List<HttpServer> ACTIVE_SERVERS = new ArrayList<>();
 
   private final HttpServer server;
   private final String url;
   private final int port;
   private final Driver.Serialization serialization;
 
-  @Parameters
+  @Parameters(name = "{0}")
   public static List<Object[]> parameters() throws Exception {
-    List<Object[]> params = new ArrayList<>();
-
-    final String[] mainArgs = { FullyRemoteJdbcMetaFactory.class.getName() };
-
-    // Bind to '0' to pluck an ephemeral port instead of expecting a certain one to be free
-
-    final HttpServer jsonServer = Main.start(mainArgs, 0, new HandlerFactory() {
-      @Override public AvaticaJsonHandler createHandler(Service service) {
-        return new AvaticaJsonHandler(service);
-      }
-    });
-    params.add(new Object[] {jsonServer, Driver.Serialization.JSON});
-    ACTIVE_SERVERS.add(jsonServer);
-
-    final HttpServer protobufServer = Main.start(mainArgs, 0, new HandlerFactory() {
-      @Override public AvaticaProtobufHandler createHandler(Service service) {
-        return new AvaticaProtobufHandler(service);
-      }
-    });
-    params.add(new Object[] {protobufServer, Driver.Serialization.PROTOBUF});
-
-    ACTIVE_SERVERS.add(protobufServer);
-
-    return params;
+    SERVERS.startServers();
+    return SERVERS.getJUnitParameters();
   }
 
-  public RemoteMetaTest(HttpServer server, Driver.Serialization serialization) {
+  public RemoteMetaTest(Driver.Serialization serialization, HttpServer server) {
     this.server = server;
     this.port = this.server.getPort();
     this.serialization = serialization;
-    url = "jdbc:avatica:remote:url=http://localhost:" + port + ";serialization="
-        + serialization.name();
+    this.url = SERVERS.getJdbcUrl(port, serialization);
   }
 
   @AfterClass public static void afterClass() throws Exception {
-    for (HttpServer server : ACTIVE_SERVERS) {
-      if (server != null) {
-        server.stop();
-      }
+    if (null != SERVERS) {
+      SERVERS.stopServers();
     }
   }
 
@@ -747,28 +715,6 @@ public class RemoteMetaTest {
       assertEquals(props, originalProps);
     }
   }
-
-  /** Factory that provides a {@link JdbcMeta}. */
-  public static class FullyRemoteJdbcMetaFactory implements Meta.Factory {
-
-    private static JdbcMeta instance = null;
-
-    private static JdbcMeta getInstance() {
-      if (instance == null) {
-        try {
-          instance = new JdbcMeta(CONNECTION_SPEC.url, CONNECTION_SPEC.username,
-              CONNECTION_SPEC.password);
-        } catch (SQLException e) {
-          throw new RuntimeException(e);
-        }
-      }
-      return instance;
-    }
-
-    @Override public Meta create(List<String> args) {
-      return getInstance();
-    }
-  }
 }
 
 // End RemoteMetaTest.java


[3/4] calcite-avatica git commit: [CALCITE-1050] Array support for Avatica

Posted by el...@apache.org.
[CALCITE-1050] Array support for Avatica

As best as possible, works around the limitations of JDBC's
Array class to handle arbitrarily nested Arrays. Nested-array
support differs from DB to DB, so functionality is primarily
driven from structure, rather than metadata.

Also, fixes the implementation of Array.getArray(long, int).The
arguments are given in terms of one-offset and number of elements,
whereas the list.subList(int, int) call is expecting zero-offset and
exclusive end-offset.

Reduce the "API" changes to avoid downstream changes in Avatica. This
"restores" some weirdness in the implementations, but simplifies the
upgrade process (probably worth it).

Closes apache/calcite-avatica#2


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

Branch: refs/heads/master
Commit: dd65a2b18b8c35cfccf1c47b6be87ea7db3ad658
Parents: 01ec883
Author: Josh Elser <el...@apache.org>
Authored: Tue Jan 12 16:40:18 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 24 17:59:17 2017 -0400

----------------------------------------------------------------------
 .../calcite/avatica/AvaticaConnection.java      |  29 +-
 .../calcite/avatica/AvaticaResultSet.java       |  39 +-
 .../org/apache/calcite/avatica/AvaticaSite.java |   1 +
 .../apache/calcite/avatica/AvaticaUtils.java    |   2 +-
 .../apache/calcite/avatica/ColumnMetaData.java  | 145 +++-
 .../java/org/apache/calcite/avatica/Meta.java   |  96 ++-
 .../org/apache/calcite/avatica/MetaImpl.java    |  45 +-
 .../apache/calcite/avatica/proto/Common.java    | 714 +++++++++++++++++--
 .../apache/calcite/avatica/remote/Service.java  |   8 +-
 .../calcite/avatica/remote/TypedValue.java      | 277 ++++++-
 .../calcite/avatica/util/AbstractCursor.java    | 137 +++-
 .../calcite/avatica/util/ArrayFactoryImpl.java  | 142 ++++
 .../apache/calcite/avatica/util/ArrayImpl.java  |  86 ++-
 .../calcite/avatica/util/PositionedCursor.java  |  15 +-
 .../org/apache/calcite/avatica/util/Unsafe.java |  11 +
 core/src/main/protobuf/common.proto             |   3 +
 .../AvaticaResultSetConversionsTest.java        |  13 -
 .../org/apache/calcite/avatica/FrameTest.java   |  28 +
 .../org/apache/calcite/avatica/RepTest.java     |  57 ++
 .../calcite/avatica/remote/TypedValueTest.java  |  29 +
 .../calcite/avatica/util/ArrayImplTest.java     | 193 +++++
 .../calcite/avatica/util/StructImplTest.java    |  92 +++
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   |   6 +-
 .../calcite/avatica/jdbc/JdbcResultSet.java     |  92 ++-
 .../calcite/avatica/RemoteDriverTest.java       |   7 +-
 .../calcite/avatica/remote/ArrayTypeTest.java   | 626 ++++++++++++++++
 .../avatica/remote/AvaticaServersForTest.java   | 141 ++++
 .../calcite/avatica/remote/RemoteMetaTest.java  |  72 +-
 28 files changed, 2802 insertions(+), 304 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java b/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
index 51649c1..21720cc 100644
--- a/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
+++ b/core/src/main/java/org/apache/calcite/avatica/AvaticaConnection.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.avatica;
 
+import org.apache.calcite.avatica.ColumnMetaData.AvaticaType;
+import org.apache.calcite.avatica.ColumnMetaData.Rep;
 import org.apache.calcite.avatica.Meta.ExecuteBatchResult;
 import org.apache.calcite.avatica.Meta.MetaResultSet;
 import org.apache.calcite.avatica.remote.KerberosConnection;
@@ -23,6 +25,7 @@ import org.apache.calcite.avatica.remote.Service;
 import org.apache.calcite.avatica.remote.Service.ErrorResponse;
 import org.apache.calcite.avatica.remote.Service.OpenConnectionRequest;
 import org.apache.calcite.avatica.remote.TypedValue;
+import org.apache.calcite.avatica.util.ArrayFactoryImpl;
 
 import java.sql.Array;
 import java.sql.Blob;
@@ -391,9 +394,29 @@ public abstract class AvaticaConnection implements Connection {
     throw helper.unsupported();
   }
 
-  public Array createArrayOf(String typeName, Object[] elements)
-      throws SQLException {
-    throw helper.unsupported();
+  public Array createArrayOf(String typeName, Object[] elements) throws SQLException {
+    @SuppressWarnings("unchecked")
+    List<Object> elementList = (List<Object>) AvaticaUtils.primitiveList(elements);
+    SqlType type;
+    try {
+      type = SqlType.valueOf(typeName);
+    } catch (IllegalArgumentException e) {
+      throw new SQLException("Could not find JDBC type for '" + typeName + "'");
+    }
+    AvaticaType avaticaType = null;
+    switch (type) {
+    case ARRAY:
+      // TODO: Nested ARRAYs
+      throw helper.createException("Cannot create an ARRAY of ARRAY's");
+    case STRUCT:
+      // TODO: ARRAYs of STRUCTs
+      throw helper.createException("Cannot create an ARRAY of STRUCT's");
+    default:
+      // This is an ARRAY, we need to use Objects, not primitives (nullable).
+      avaticaType = ColumnMetaData.scalar(type.id, typeName, Rep.nonPrimitiveRepOf(type));
+    }
+    ArrayFactoryImpl arrayFactory = new ArrayFactoryImpl(getTimeZone());
+    return arrayFactory.createArray(avaticaType, elementList);
   }
 
   public Struct createStruct(String typeName, Object[] attributes)

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java b/core/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java
index 51bf8fa..e78b6e9 100644
--- a/core/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java
+++ b/core/src/main/java/org/apache/calcite/avatica/AvaticaResultSet.java
@@ -17,7 +17,7 @@
 package org.apache.calcite.avatica;
 
 import org.apache.calcite.avatica.remote.TypedValue;
-import org.apache.calcite.avatica.util.ArrayImpl;
+import org.apache.calcite.avatica.util.ArrayFactoryImpl;
 import org.apache.calcite.avatica.util.Cursor;
 
 import java.io.InputStream;
@@ -39,6 +39,7 @@ import java.sql.SQLXML;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.Calendar;
+import java.util.Collections;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -48,7 +49,7 @@ import java.util.TimeZone;
  * Implementation of {@link java.sql.ResultSet}
  * for the Avatica engine.
  */
-public class AvaticaResultSet implements ResultSet, ArrayImpl.Factory {
+public class AvaticaResultSet extends ArrayFactoryImpl implements ResultSet {
   protected final AvaticaStatement statement;
   protected final QueryState state;
   protected final Meta.Signature signature;
@@ -77,16 +78,25 @@ public class AvaticaResultSet implements ResultSet, ArrayImpl.Factory {
       ResultSetMetaData resultSetMetaData,
       TimeZone timeZone,
       Meta.Frame firstFrame) {
+    super(timeZone);
     this.statement = statement;
     this.state = state;
     this.signature = signature;
     this.firstFrame = firstFrame;
     this.columnMetaDataList = signature.columns;
-    this.type = statement.resultSetType;
-    this.concurrency = statement.resultSetConcurrency;
-    this.holdability = statement.resultSetHoldability;
-    this.fetchSize = statement.getFetchSize();
-    this.fetchDirection = statement.getFetchDirection();
+    if (null != statement) {
+      this.type = statement.resultSetType;
+      this.concurrency = statement.resultSetConcurrency;
+      this.holdability = statement.resultSetHoldability;
+      this.fetchSize = statement.getFetchSize();
+      this.fetchDirection = statement.getFetchDirection();
+    } else {
+      this.type = 0;
+      this.concurrency = 0;
+      this.holdability = 0;
+      this.fetchSize = AvaticaStatement.DEFAULT_FETCH_SIZE;
+      this.fetchDirection = 0;
+    }
     this.resultSetMetaData = resultSetMetaData;
     this.localCalendar = Calendar.getInstance(timeZone, Locale.ROOT);
   }
@@ -190,10 +200,9 @@ public class AvaticaResultSet implements ResultSet, ArrayImpl.Factory {
    * @throws SQLException if execute fails for some reason.
    */
   protected AvaticaResultSet execute() throws SQLException {
-    final List<TypedValue> parameterValues = statement.getBoundParameterValues();
     final Iterable<Object> iterable1 =
         statement.connection.meta.createIterable(statement.handle, state, signature,
-            parameterValues, firstFrame);
+            Collections.<TypedValue>emptyList(), firstFrame);
     this.cursor = MetaImpl.createCursor(signature.cursorFactory, iterable1);
     this.accessorList =
         cursor.createAccessors(columnMetaDataList, localCalendar, this);
@@ -202,7 +211,7 @@ public class AvaticaResultSet implements ResultSet, ArrayImpl.Factory {
     return this;
   }
 
-  protected AvaticaResultSet execute2(Cursor cursor,
+  public AvaticaResultSet execute2(Cursor cursor,
       List<ColumnMetaData> columnMetaDataList) {
     this.cursor = cursor;
     this.accessorList =
@@ -212,9 +221,11 @@ public class AvaticaResultSet implements ResultSet, ArrayImpl.Factory {
     return this;
   }
 
-  public ResultSet create(ColumnMetaData.AvaticaType elementType,
-      Iterable<Object> iterable) {
-    throw new UnsupportedOperationException();
+  /**
+   * Returns the calendar used by this result set. Not a jdbc method.
+   */
+  public Calendar getLocalCalendar() {
+    return localCalendar;
   }
 
   public boolean next() throws SQLException {
@@ -222,7 +233,7 @@ public class AvaticaResultSet implements ResultSet, ArrayImpl.Factory {
     if (isClosed()) {
       throw new SQLException("next() called on closed cursor");
     }
-    if (statement.cancelFlag.get()) {
+    if (null != statement && statement.cancelFlag.get()) {
       throw new SQLException("Statement canceled");
     }
     if (cursor.next()) {

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/AvaticaSite.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/AvaticaSite.java b/core/src/main/java/org/apache/calcite/avatica/AvaticaSite.java
index 0e23866..ece9e04 100644
--- a/core/src/main/java/org/apache/calcite/avatica/AvaticaSite.java
+++ b/core/src/main/java/org/apache/calcite/avatica/AvaticaSite.java
@@ -398,6 +398,7 @@ public class AvaticaSite {
   }
 
   public void setArray(Array x) {
+    slots[index] = wrap(ColumnMetaData.Rep.ARRAY, x);
   }
 
   public void setNull(int sqlType, String typeName) {

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/AvaticaUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/AvaticaUtils.java b/core/src/main/java/org/apache/calcite/avatica/AvaticaUtils.java
index 56a23c8..45e8236 100644
--- a/core/src/main/java/org/apache/calcite/avatica/AvaticaUtils.java
+++ b/core/src/main/java/org/apache/calcite/avatica/AvaticaUtils.java
@@ -133,7 +133,7 @@ public class AvaticaUtils {
    */
   public static List<?> primitiveList(final Object array) {
     // REVIEW: A per-type list might be more efficient. (Or might not.)
-    return new AbstractList() {
+    return new AbstractList<Object>() {
       public Object get(int index) {
         return java.lang.reflect.Array.get(array, index);
       }

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java b/core/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
index 401070e..0f00fd2 100644
--- a/core/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
+++ b/core/src/main/java/org/apache/calcite/avatica/ColumnMetaData.java
@@ -316,40 +316,41 @@ public class ColumnMetaData {
    * a {@link java.sql.Date} might be represented as a {@link #PRIMITIVE_INT}
    * if not nullable, or a {@link #JAVA_SQL_DATE}. */
   public enum Rep {
-    PRIMITIVE_BOOLEAN(boolean.class),
-    PRIMITIVE_BYTE(byte.class),
-    PRIMITIVE_CHAR(char.class),
-    PRIMITIVE_SHORT(short.class),
-    PRIMITIVE_INT(int.class),
-    PRIMITIVE_LONG(long.class),
-    PRIMITIVE_FLOAT(float.class),
-    PRIMITIVE_DOUBLE(double.class),
-    BOOLEAN(Boolean.class),
-    BYTE(Byte.class),
-    CHARACTER(Character.class),
-    SHORT(Short.class),
-    INTEGER(Integer.class),
-    LONG(Long.class),
-    FLOAT(Float.class),
-    DOUBLE(Double.class),
-    JAVA_SQL_TIME(Time.class),
-    JAVA_SQL_TIMESTAMP(Timestamp.class),
-    JAVA_SQL_DATE(java.sql.Date.class),
-    JAVA_UTIL_DATE(java.util.Date.class),
-    BYTE_STRING(ByteString.class),
-    STRING(String.class),
+    PRIMITIVE_BOOLEAN(boolean.class, Types.BOOLEAN),
+    PRIMITIVE_BYTE(byte.class, Types.TINYINT),
+    PRIMITIVE_CHAR(char.class, Types.CHAR),
+    PRIMITIVE_SHORT(short.class, Types.SMALLINT),
+    PRIMITIVE_INT(int.class, Types.INTEGER),
+    PRIMITIVE_LONG(long.class, Types.BIGINT),
+    PRIMITIVE_FLOAT(float.class, Types.FLOAT),
+    PRIMITIVE_DOUBLE(double.class, Types.DOUBLE),
+    BOOLEAN(Boolean.class, Types.BOOLEAN),
+    BYTE(Byte.class, Types.TINYINT),
+    CHARACTER(Character.class, Types.CHAR),
+    SHORT(Short.class, Types.SMALLINT),
+    INTEGER(Integer.class, Types.INTEGER),
+    LONG(Long.class, Types.BIGINT),
+    FLOAT(Float.class, Types.FLOAT),
+    DOUBLE(Double.class, Types.DOUBLE),
+    JAVA_SQL_TIME(Time.class, Types.TIME),
+    JAVA_SQL_TIMESTAMP(Timestamp.class, Types.TIMESTAMP),
+    JAVA_SQL_DATE(java.sql.Date.class, Types.DATE),
+    JAVA_UTIL_DATE(java.util.Date.class, Types.DATE),
+    BYTE_STRING(ByteString.class, Types.VARBINARY),
+    STRING(String.class, Types.VARCHAR),
 
     /** Values are represented as some sub-class of {@link Number}.
      * The JSON encoding does this. */
-    NUMBER(Number.class),
+    NUMBER(Number.class, Types.NUMERIC),
 
-    ARRAY(Array.class),
-    MULTISET(List.class),
-    STRUCT(Struct.class),
+    ARRAY(Array.class, Types.ARRAY),
+    MULTISET(List.class, Types.JAVA_OBJECT),
+    STRUCT(Struct.class, Types.JAVA_OBJECT),
 
-    OBJECT(Object.class);
+    OBJECT(Object.class, Types.JAVA_OBJECT);
 
     public final Class clazz;
+    public final int typeId;
 
     public static final Map<Class, Rep> VALUE_MAP;
 
@@ -358,11 +359,13 @@ public class ColumnMetaData {
       for (Rep rep : values()) {
         builder.put(rep.clazz, rep);
       }
+      builder.put(byte[].class, BYTE_STRING);
       VALUE_MAP = Collections.unmodifiableMap(builder);
     }
 
-    Rep(Class clazz) {
+    Rep(Class clazz, int typeId) {
       this.clazz = clazz;
+      this.typeId = typeId;
     }
 
     public static Rep of(Type clazz) {
@@ -429,7 +432,10 @@ public class ColumnMetaData {
     }
 
     public static Rep fromProto(Common.Rep proto) {
-      if (Common.Rep.BIG_DECIMAL == proto) {
+      if (Common.Rep.UNRECOGNIZED == proto) {
+        // Un-set in the message, treat it as null
+        return null;
+      } else if (Common.Rep.BIG_DECIMAL == proto) {
         // BIG_DECIMAL has to come back as a NUMBER
         return Rep.NUMBER;
       } else if (Common.Rep.NULL == proto) {
@@ -437,6 +443,68 @@ public class ColumnMetaData {
       }
       return Rep.valueOf(proto.name());
     }
+
+    /**
+     * Computes the given JDBC type for a primitive to the corresponding {@link Rep} for the
+     * equivalent Object type. If the provided type is not for a primitive, a {@link Rep} for the
+     * provided Object is returned.
+     *
+     * @param type The type of a value (based on {@link java.sql.Types}).
+     * @return The corresponding non-primitive {@link Rep} for the given {@code type}.
+     */
+    public static ColumnMetaData.Rep nonPrimitiveRepOf(SqlType type) {
+      if (null == type) {
+        throw new NullPointerException();
+      }
+      if (boolean.class == type.clazz) {
+        return ColumnMetaData.Rep.BOOLEAN;
+      } else if (byte.class == type.clazz) {
+        return ColumnMetaData.Rep.BYTE;
+      } else  if (char.class == type.clazz) {
+        return ColumnMetaData.Rep.CHARACTER;
+      } else if (short.class == type.clazz) {
+        return ColumnMetaData.Rep.SHORT;
+      } else if (int.class == type.clazz) {
+        return ColumnMetaData.Rep.INTEGER;
+      } else if (long.class == type.clazz) {
+        return ColumnMetaData.Rep.LONG;
+      } else if (float.class == type.clazz) {
+        return ColumnMetaData.Rep.FLOAT;
+      } else if (double.class == type.clazz) {
+        return ColumnMetaData.Rep.DOUBLE;
+      }
+      return ColumnMetaData.Rep.of(type.clazz);
+    }
+
+    /**
+     * Computes the given JDBC type into the {@link Rep} for the wire (serial) form of that type.
+     *
+     * @param type The type of a value (based on {@link java.sql.Types}).
+     * @return The corresponding {@link Rep} for the serial form of the {@code type}.
+     */
+    public static ColumnMetaData.Rep serialRepOf(SqlType type) {
+      if (null == type) {
+        throw new NullPointerException();
+      }
+      if (boolean.class == type.internal) {
+        return ColumnMetaData.Rep.BOOLEAN;
+      } else if (byte.class == type.internal) {
+        return ColumnMetaData.Rep.BYTE;
+      } else  if (char.class == type.internal) {
+        return ColumnMetaData.Rep.CHARACTER;
+      } else if (short.class == type.internal) {
+        return ColumnMetaData.Rep.SHORT;
+      } else if (int.class == type.internal) {
+        return ColumnMetaData.Rep.INTEGER;
+      } else if (long.class == type.internal) {
+        return ColumnMetaData.Rep.LONG;
+      } else if (float.class == type.internal) {
+        return ColumnMetaData.Rep.FLOAT;
+      } else if (double.class == type.internal) {
+        return ColumnMetaData.Rep.DOUBLE;
+      }
+      return ColumnMetaData.Rep.of(type.internal);
+    }
   }
 
   /** Base class for a column type. */
@@ -465,6 +533,10 @@ public class ColumnMetaData {
       return SqlType.valueOf(id).boxedClass().getName();
     }
 
+    public String getName() {
+      return name;
+    }
+
     public AvaticaType setRep(Rep rep) {
       throw new UnsupportedOperationException();
     }
@@ -563,7 +635,7 @@ public class ColumnMetaData {
 
   /** Array type. */
   public static class ArrayType extends AvaticaType {
-    public final AvaticaType component;
+    private AvaticaType component;
 
     /**
      * Not for public use. Use {@link ColumnMetaData#array(AvaticaType, String, Rep)}.
@@ -575,6 +647,19 @@ public class ColumnMetaData {
       this.component = component;
     }
 
+    /**
+     * Updates the component of {@code this} to the given value. This is necessary to provide as
+     * accurate-as-possible of an {@code ArrayType} in the {@code Signature}. It cannot be done
+     * at initial construction of this object.
+     */
+    public void updateComponentType(AvaticaType component) {
+      this.component = Objects.requireNonNull(component);
+    }
+
+    public AvaticaType getComponent() {
+      return component;
+    }
+
     @Override public Common.AvaticaType toProto() {
       Common.AvaticaType.Builder builder = Common.AvaticaType.newBuilder(super.toProto());
 

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/Meta.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/Meta.java b/core/src/main/java/org/apache/calcite/avatica/Meta.java
index 7df9ade..a71214d 100644
--- a/core/src/main/java/org/apache/calcite/avatica/Meta.java
+++ b/core/src/main/java/org/apache/calcite/avatica/Meta.java
@@ -203,7 +203,7 @@ public interface Meta {
    * requires to be not null; derived classes may instead choose to execute the
    * relational expression in {@code signature}. */
   Iterable<Object> createIterable(StatementHandle stmt, QueryState state, Signature signature,
-      List<TypedValue> parameterValues, Frame firstFrame);
+      List<TypedValue> parameters, Frame firstFrame);
 
   /** Prepares a statement.
    *
@@ -929,47 +929,55 @@ public interface Meta {
           continue;
         }
 
+        final Common.Row.Builder rowBuilder = Common.Row.newBuilder();
         if (row instanceof Object[]) {
-          final Common.Row.Builder rowBuilder = Common.Row.newBuilder();
-
+          // If only Object[] was also Iterable.
           for (Object element : (Object[]) row) {
-            final Common.ColumnValue.Builder columnBuilder = Common.ColumnValue.newBuilder();
-
-            if (element instanceof List) {
-              columnBuilder.setHasArrayValue(true);
-              List<?> list = (List<?>) element;
-              // Add each element in the list/array to the column's value
-              for (Object listItem : list) {
-                final Common.TypedValue scalarListItem = serializeScalar(listItem);
-                columnBuilder.addArrayValue(scalarListItem);
-                // Add the deprecated 'value' repeated attribute for backwards compat
-                columnBuilder.addValue(scalarListItem);
-              }
-            } else {
-              // The default value, but still explicit.
-              columnBuilder.setHasArrayValue(false);
-              // Only one value for this column, a scalar.
-              final Common.TypedValue scalarVal = serializeScalar(element);
-              columnBuilder.setScalarValue(scalarVal);
-              // Add the deprecated 'value' repeated attribute for backwards compat
-              columnBuilder.addValue(scalarVal);
-            }
-
-            // Add value to row
-            rowBuilder.addValue(columnBuilder.build());
+            parseColumn(rowBuilder, element);
+          }
+        } else if (row instanceof Iterable) {
+          for (Object element : (Iterable<?>) row) {
+            parseColumn(rowBuilder, element);
           }
-
-          // Collect all rows
-          builder.addRows(rowBuilder.build());
         } else {
           // Can a "row" be a primitive? A struct? Only an Array?
           throw new RuntimeException("Only arrays are supported");
         }
+
+        // Collect all rows
+        builder.addRows(rowBuilder.build());
       }
 
       return builder.build();
     }
 
+    static void parseColumn(Common.Row.Builder rowBuilder, Object column) {
+      final Common.ColumnValue.Builder columnBuilder = Common.ColumnValue.newBuilder();
+
+      if (column instanceof List) {
+        columnBuilder.setHasArrayValue(true);
+        List<?> list = (List<?>) column;
+        // Add each element in the list/array to the column's value
+        for (Object listItem : list) {
+          final Common.TypedValue scalarListItem = serializeScalar(listItem);
+          columnBuilder.addArrayValue(scalarListItem);
+          // Add the deprecated 'value' repeated attribute for backwards compat
+          columnBuilder.addValue(scalarListItem);
+        }
+      } else {
+        // The default value, but still explicit.
+        columnBuilder.setHasArrayValue(false);
+        // Only one value for this column, a scalar.
+        final Common.TypedValue scalarVal = serializeScalar(column);
+        columnBuilder.setScalarValue(scalarVal);
+        // Add the deprecated 'value' repeated attribute for backwards compat
+        columnBuilder.addValue(scalarVal);
+      }
+
+      // Add value to row
+      rowBuilder.addValue(columnBuilder.build());
+    }
+
     static Common.TypedValue serializeScalar(Object element) {
       final Common.TypedValue.Builder valueBuilder = Common.TypedValue.newBuilder();
 
@@ -1046,10 +1054,17 @@ public interface Meta {
       validateColumnValue(column);
 
       if (!column.hasField(SCALAR_VALUE_DESCRIPTOR)) {
-        // Array
+        // The column in this row is an Array (has multiple values)
         List<Object> array = new ArrayList<>(column.getArrayValueCount());
         for (Common.TypedValue arrayValue : column.getArrayValueList()) {
-          array.add(deserializeScalarValue(arrayValue));
+          // Duplicative because of the ColumnValue/TypedValue difference.
+          if (Common.Rep.ARRAY == arrayValue.getType()) {
+            // Each element in this Array is an Array.
+            array.add(parseArray(arrayValue));
+          } else {
+            // The array element is a scalar.
+            array.add(deserializeScalarValue(arrayValue));
+          }
         }
         return array;
       } else {
@@ -1059,6 +1074,23 @@ public interface Meta {
     }
 
     /**
+     * Recursively parses a TypedValue while it is an array.
+     */
+    static Object parseArray(Common.TypedValue array) {
+      List<Object> convertedArray = new ArrayList<>(array.getArrayValueCount());
+      for (Common.TypedValue arrayElement : array.getArrayValueList()) {
+        if (Common.Rep.ARRAY == arrayElement.getType()) {
+          // Recurse
+          convertedArray.add(parseArray(arrayElement));
+        } else {
+          // The component type of this array is a scalar.
+          convertedArray.add(deserializeScalarValue(arrayElement));
+        }
+      }
+      return convertedArray;
+    }
+
+    /**
      * Verifies that a ColumnValue has only a scalar or array value, not both and not neither.
      *
      * @param column The protobuf ColumnValue object

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/MetaImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/MetaImpl.java b/core/src/main/java/org/apache/calcite/avatica/MetaImpl.java
index 8870e0c..f663ebb 100644
--- a/core/src/main/java/org/apache/calcite/avatica/MetaImpl.java
+++ b/core/src/main/java/org/apache/calcite/avatica/MetaImpl.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.avatica;
 
+import org.apache.calcite.avatica.ColumnMetaData.AvaticaType;
 import org.apache.calcite.avatica.remote.TypedValue;
 import org.apache.calcite.avatica.util.ArrayIteratorCursor;
 import org.apache.calcite.avatica.util.Cursor;
@@ -222,11 +223,18 @@ public abstract class MetaImpl implements Meta {
         Frame.EMPTY);
   }
 
+  private static int intForColumnNullable(boolean nullable) {
+    return nullable ? DatabaseMetaData.columnNullable : DatabaseMetaData.columnNoNulls;
+  }
+
   public static ColumnMetaData columnMetaData(String name, int index,
       Class<?> type, boolean columnNullable) {
-    return columnMetaData(name, index, type, columnNullable
-        ? DatabaseMetaData.columnNullable
-        : DatabaseMetaData.columnNoNulls);
+    return columnMetaData(name, index, type, intForColumnNullable(columnNullable));
+  }
+
+  public static ColumnMetaData columnMetaData(String name, int index, AvaticaType type,
+      boolean columnNullable) {
+    return columnMetaData(name, index, type, intForColumnNullable(columnNullable));
   }
 
   public static ColumnMetaData columnMetaData(String name, int index,
@@ -236,12 +244,17 @@ public abstract class MetaImpl implements Meta {
         ColumnMetaData.Rep.VALUE_MAP.get(type);
     ColumnMetaData.AvaticaType scalarType =
         ColumnMetaData.scalar(pair.sqlType, pair.sqlTypeName, rep);
+    return columnMetaData(name, index, scalarType, columnNullable);
+  }
+
+  public static ColumnMetaData columnMetaData(String name, int index, AvaticaType type,
+      int columnNullable) {
     return new ColumnMetaData(
         index, false, true, false, false,
         columnNullable,
         true, -1, name, name, null,
-        0, 0, null, null, scalarType, true, false, false,
-        scalarType.columnClassName());
+        0, 0, null, null, type, true, false, false,
+        type.columnClassName());
   }
 
   protected static ColumnMetaData.StructType fieldMetaData(Class<?> clazz) {
@@ -1448,6 +1461,8 @@ public abstract class MetaImpl implements Meta {
 
   @Override public Iterable<Object> createIterable(StatementHandle handle, QueryState state,
       Signature signature, List<TypedValue> parameterValues, Frame firstFrame) {
+    // `parameterValues` is intentionally unusued (in method signature for historic reasons)
+    // Left to preserve API compatibility with Calcite
     if (firstFrame != null && firstFrame.done) {
       return firstFrame.rows;
     }
@@ -1457,8 +1472,7 @@ public abstract class MetaImpl implements Meta {
     } catch (SQLException e) {
       throw new RuntimeException(e);
     }
-    return new FetchIterable(stmt, state,
-        firstFrame, parameterValues);
+    return new FetchIterable(stmt, state, firstFrame);
   }
 
   public Frame fetch(AvaticaStatement stmt, List<TypedValue> parameterValues,
@@ -1527,18 +1541,15 @@ public abstract class MetaImpl implements Meta {
     private final AvaticaStatement stmt;
     private final QueryState state;
     private final Frame firstFrame;
-    private final List<TypedValue> parameterValues;
 
-    public FetchIterable(AvaticaStatement stmt, QueryState state, Frame firstFrame,
-        List<TypedValue> parameterValues) {
+    public FetchIterable(AvaticaStatement stmt, QueryState state, Frame firstFrame) {
       this.stmt = stmt;
       this.state = state;
       this.firstFrame = firstFrame;
-      this.parameterValues = parameterValues;
     }
 
     public Iterator<Object> iterator() {
-      return new FetchIterator(stmt, state, firstFrame, parameterValues);
+      return new FetchIterator(stmt, state, firstFrame);
     }
   }
 
@@ -1548,16 +1559,11 @@ public abstract class MetaImpl implements Meta {
     private final QueryState state;
     private Frame frame;
     private Iterator<Object> rows;
-    private List<TypedValue> parameterValues;
-    private List<TypedValue> originalParameterValues;
     private long currentOffset = 0;
 
-    public FetchIterator(AvaticaStatement stmt, QueryState state, Frame firstFrame,
-        List<TypedValue> parameterValues) {
+    public FetchIterator(AvaticaStatement stmt, QueryState state, Frame firstFrame) {
       this.stmt = stmt;
       this.state = state;
-      this.parameterValues = parameterValues;
-      this.originalParameterValues = parameterValues;
       if (firstFrame == null) {
         frame = Frame.MORE;
         rows = EmptyIterator.INSTANCE;
@@ -1620,7 +1626,6 @@ public abstract class MetaImpl implements Meta {
           // Kick back to the top to try to fetch again (in both branches)
           continue;
         }
-        parameterValues = null; // don't execute next time
         if (frame == null) {
           rows = null;
           break;
@@ -1632,8 +1637,6 @@ public abstract class MetaImpl implements Meta {
     }
 
     private void resetStatement() {
-      // If we have to reset the statement, we need to reset the parameterValues too
-      parameterValues = originalParameterValues;
       // Defer to the statement to reset itself
       stmt.resetStatement();
     }

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/proto/Common.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/proto/Common.java b/core/src/main/java/org/apache/calcite/avatica/proto/Common.java
index 892608e..50450f4 100644
--- a/core/src/main/java/org/apache/calcite/avatica/proto/Common.java
+++ b/core/src/main/java/org/apache/calcite/avatica/proto/Common.java
@@ -14535,6 +14535,67 @@ public final class Common {
      * <code>optional bool null = 7;</code>
      */
     boolean getNull();
+
+    /**
+     * <pre>
+     * The Array
+     * </pre>
+     *
+     * <code>repeated .TypedValue array_value = 8;</code>
+     */
+    java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> 
+        getArrayValueList();
+    /**
+     * <pre>
+     * The Array
+     * </pre>
+     *
+     * <code>repeated .TypedValue array_value = 8;</code>
+     */
+    org.apache.calcite.avatica.proto.Common.TypedValue getArrayValue(int index);
+    /**
+     * <pre>
+     * The Array
+     * </pre>
+     *
+     * <code>repeated .TypedValue array_value = 8;</code>
+     */
+    int getArrayValueCount();
+    /**
+     * <pre>
+     * The Array
+     * </pre>
+     *
+     * <code>repeated .TypedValue array_value = 8;</code>
+     */
+    java.util.List<? extends org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+        getArrayValueOrBuilderList();
+    /**
+     * <pre>
+     * The Array
+     * </pre>
+     *
+     * <code>repeated .TypedValue array_value = 8;</code>
+     */
+    org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getArrayValueOrBuilder(
+        int index);
+
+    /**
+     * <pre>
+     * If an Array, the representation for the array values
+     * </pre>
+     *
+     * <code>optional .Rep component_type = 9;</code>
+     */
+    int getComponentTypeValue();
+    /**
+     * <pre>
+     * If an Array, the representation for the array values
+     * </pre>
+     *
+     * <code>optional .Rep component_type = 9;</code>
+     */
+    org.apache.calcite.avatica.proto.Common.Rep getComponentType();
   }
   /**
    * <pre>
@@ -14559,6 +14620,8 @@ public final class Common {
       bytesValue_ = com.google.protobuf.ByteString.EMPTY;
       doubleValue_ = 0D;
       null_ = false;
+      arrayValue_ = java.util.Collections.emptyList();
+      componentType_ = 0;
     }
 
     @java.lang.Override
@@ -14623,6 +14686,21 @@ public final class Common {
               null_ = input.readBool();
               break;
             }
+            case 66: {
+              if (!((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
+                arrayValue_ = new java.util.ArrayList<org.apache.calcite.avatica.proto.Common.TypedValue>();
+                mutable_bitField0_ |= 0x00000080;
+              }
+              arrayValue_.add(
+                  input.readMessage(org.apache.calcite.avatica.proto.Common.TypedValue.parser(), extensionRegistry));
+              break;
+            }
+            case 72: {
+              int rawValue = input.readEnum();
+
+              componentType_ = rawValue;
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -14631,6 +14709,9 @@ public final class Common {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
+        if (((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
+          arrayValue_ = java.util.Collections.unmodifiableList(arrayValue_);
+        }
         makeExtensionsImmutable();
       }
     }
@@ -14646,6 +14727,7 @@ public final class Common {
               org.apache.calcite.avatica.proto.Common.TypedValue.class, org.apache.calcite.avatica.proto.Common.TypedValue.Builder.class);
     }
 
+    private int bitField0_;
     public static final int TYPE_FIELD_NUMBER = 1;
     private int type_;
     /**
@@ -14777,6 +14859,85 @@ public final class Common {
       return null_;
     }
 
+    public static final int ARRAY_VALUE_FIELD_NUMBER = 8;
+    private java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> arrayValue_;
+    /**
+     * <pre>
+     * The Array
+     * </pre>
+     *
+     * <code>repeated .TypedValue array_value = 8;</code>
+     */
+    public java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> getArrayValueList() {
+      return arrayValue_;
+    }
+    /**
+     * <pre>
+     * The Array
+     * </pre>
+     *
+     * <code>repeated .TypedValue array_value = 8;</code>
+     */
+    public java.util.List<? extends org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+        getArrayValueOrBuilderList() {
+      return arrayValue_;
+    }
+    /**
+     * <pre>
+     * The Array
+     * </pre>
+     *
+     * <code>repeated .TypedValue array_value = 8;</code>
+     */
+    public int getArrayValueCount() {
+      return arrayValue_.size();
+    }
+    /**
+     * <pre>
+     * The Array
+     * </pre>
+     *
+     * <code>repeated .TypedValue array_value = 8;</code>
+     */
+    public org.apache.calcite.avatica.proto.Common.TypedValue getArrayValue(int index) {
+      return arrayValue_.get(index);
+    }
+    /**
+     * <pre>
+     * The Array
+     * </pre>
+     *
+     * <code>repeated .TypedValue array_value = 8;</code>
+     */
+    public org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getArrayValueOrBuilder(
+        int index) {
+      return arrayValue_.get(index);
+    }
+
+    public static final int COMPONENT_TYPE_FIELD_NUMBER = 9;
+    private int componentType_;
+    /**
+     * <pre>
+     * If an Array, the representation for the array values
+     * </pre>
+     *
+     * <code>optional .Rep component_type = 9;</code>
+     */
+    public int getComponentTypeValue() {
+      return componentType_;
+    }
+    /**
+     * <pre>
+     * If an Array, the representation for the array values
+     * </pre>
+     *
+     * <code>optional .Rep component_type = 9;</code>
+     */
+    public org.apache.calcite.avatica.proto.Common.Rep getComponentType() {
+      org.apache.calcite.avatica.proto.Common.Rep result = org.apache.calcite.avatica.proto.Common.Rep.valueOf(componentType_);
+      return result == null ? org.apache.calcite.avatica.proto.Common.Rep.UNRECOGNIZED : result;
+    }
+
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
@@ -14810,6 +14971,12 @@ public final class Common {
       if (null_ != false) {
         output.writeBool(7, null_);
       }
+      for (int i = 0; i < arrayValue_.size(); i++) {
+        output.writeMessage(8, arrayValue_.get(i));
+      }
+      if (componentType_ != org.apache.calcite.avatica.proto.Common.Rep.PRIMITIVE_BOOLEAN.getNumber()) {
+        output.writeEnum(9, componentType_);
+      }
     }
 
     public int getSerializedSize() {
@@ -14844,6 +15011,14 @@ public final class Common {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(7, null_);
       }
+      for (int i = 0; i < arrayValue_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(8, arrayValue_.get(i));
+      }
+      if (componentType_ != org.apache.calcite.avatica.proto.Common.Rep.PRIMITIVE_BOOLEAN.getNumber()) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(9, componentType_);
+      }
       memoizedSize = size;
       return size;
     }
@@ -14875,6 +15050,9 @@ public final class Common {
               other.getDoubleValue()));
       result = result && (getNull()
           == other.getNull());
+      result = result && getArrayValueList()
+          .equals(other.getArrayValueList());
+      result = result && componentType_ == other.componentType_;
       return result;
     }
 
@@ -14903,6 +15081,12 @@ public final class Common {
       hash = (37 * hash) + NULL_FIELD_NUMBER;
       hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
           getNull());
+      if (getArrayValueCount() > 0) {
+        hash = (37 * hash) + ARRAY_VALUE_FIELD_NUMBER;
+        hash = (53 * hash) + getArrayValueList().hashCode();
+      }
+      hash = (37 * hash) + COMPONENT_TYPE_FIELD_NUMBER;
+      hash = (53 * hash) + componentType_;
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -15021,6 +15205,7 @@ public final class Common {
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
+          getArrayValueFieldBuilder();
         }
       }
       public Builder clear() {
@@ -15039,6 +15224,14 @@ public final class Common {
 
         null_ = false;
 
+        if (arrayValueBuilder_ == null) {
+          arrayValue_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000080);
+        } else {
+          arrayValueBuilder_.clear();
+        }
+        componentType_ = 0;
+
         return this;
       }
 
@@ -15061,6 +15254,8 @@ public final class Common {
 
       public org.apache.calcite.avatica.proto.Common.TypedValue buildPartial() {
         org.apache.calcite.avatica.proto.Common.TypedValue result = new org.apache.calcite.avatica.proto.Common.TypedValue(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
         result.type_ = type_;
         result.boolValue_ = boolValue_;
         result.stringValue_ = stringValue_;
@@ -15068,6 +15263,17 @@ public final class Common {
         result.bytesValue_ = bytesValue_;
         result.doubleValue_ = doubleValue_;
         result.null_ = null_;
+        if (arrayValueBuilder_ == null) {
+          if (((bitField0_ & 0x00000080) == 0x00000080)) {
+            arrayValue_ = java.util.Collections.unmodifiableList(arrayValue_);
+            bitField0_ = (bitField0_ & ~0x00000080);
+          }
+          result.arrayValue_ = arrayValue_;
+        } else {
+          result.arrayValue_ = arrayValueBuilder_.build();
+        }
+        result.componentType_ = componentType_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -15131,6 +15337,35 @@ public final class Common {
         if (other.getNull() != false) {
           setNull(other.getNull());
         }
+        if (arrayValueBuilder_ == null) {
+          if (!other.arrayValue_.isEmpty()) {
+            if (arrayValue_.isEmpty()) {
+              arrayValue_ = other.arrayValue_;
+              bitField0_ = (bitField0_ & ~0x00000080);
+            } else {
+              ensureArrayValueIsMutable();
+              arrayValue_.addAll(other.arrayValue_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.arrayValue_.isEmpty()) {
+            if (arrayValueBuilder_.isEmpty()) {
+              arrayValueBuilder_.dispose();
+              arrayValueBuilder_ = null;
+              arrayValue_ = other.arrayValue_;
+              bitField0_ = (bitField0_ & ~0x00000080);
+              arrayValueBuilder_ = 
+                com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getArrayValueFieldBuilder() : null;
+            } else {
+              arrayValueBuilder_.addAllMessages(other.arrayValue_);
+            }
+          }
+        }
+        if (other.componentType_ != 0) {
+          setComponentTypeValue(other.getComponentTypeValue());
+        }
         onChanged();
         return this;
       }
@@ -15156,6 +15391,7 @@ public final class Common {
         }
         return this;
       }
+      private int bitField0_;
 
       private int type_ = 0;
       /**
@@ -15502,6 +15738,382 @@ public final class Common {
         onChanged();
         return this;
       }
+
+      private java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> arrayValue_ =
+        java.util.Collections.emptyList();
+      private void ensureArrayValueIsMutable() {
+        if (!((bitField0_ & 0x00000080) == 0x00000080)) {
+          arrayValue_ = new java.util.ArrayList<org.apache.calcite.avatica.proto.Common.TypedValue>(arrayValue_);
+          bitField0_ |= 0x00000080;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.calcite.avatica.proto.Common.TypedValue, org.apache.calcite.avatica.proto.Common.TypedValue.Builder, org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> arrayValueBuilder_;
+
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue> getArrayValueList() {
+        if (arrayValueBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(arrayValue_);
+        } else {
+          return arrayValueBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public int getArrayValueCount() {
+        if (arrayValueBuilder_ == null) {
+          return arrayValue_.size();
+        } else {
+          return arrayValueBuilder_.getCount();
+        }
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue getArrayValue(int index) {
+        if (arrayValueBuilder_ == null) {
+          return arrayValue_.get(index);
+        } else {
+          return arrayValueBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public Builder setArrayValue(
+          int index, org.apache.calcite.avatica.proto.Common.TypedValue value) {
+        if (arrayValueBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureArrayValueIsMutable();
+          arrayValue_.set(index, value);
+          onChanged();
+        } else {
+          arrayValueBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public Builder setArrayValue(
+          int index, org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
+        if (arrayValueBuilder_ == null) {
+          ensureArrayValueIsMutable();
+          arrayValue_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          arrayValueBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public Builder addArrayValue(org.apache.calcite.avatica.proto.Common.TypedValue value) {
+        if (arrayValueBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureArrayValueIsMutable();
+          arrayValue_.add(value);
+          onChanged();
+        } else {
+          arrayValueBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public Builder addArrayValue(
+          int index, org.apache.calcite.avatica.proto.Common.TypedValue value) {
+        if (arrayValueBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureArrayValueIsMutable();
+          arrayValue_.add(index, value);
+          onChanged();
+        } else {
+          arrayValueBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public Builder addArrayValue(
+          org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
+        if (arrayValueBuilder_ == null) {
+          ensureArrayValueIsMutable();
+          arrayValue_.add(builderForValue.build());
+          onChanged();
+        } else {
+          arrayValueBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public Builder addArrayValue(
+          int index, org.apache.calcite.avatica.proto.Common.TypedValue.Builder builderForValue) {
+        if (arrayValueBuilder_ == null) {
+          ensureArrayValueIsMutable();
+          arrayValue_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          arrayValueBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public Builder addAllArrayValue(
+          java.lang.Iterable<? extends org.apache.calcite.avatica.proto.Common.TypedValue> values) {
+        if (arrayValueBuilder_ == null) {
+          ensureArrayValueIsMutable();
+          com.google.protobuf.AbstractMessageLite.Builder.addAll(
+              values, arrayValue_);
+          onChanged();
+        } else {
+          arrayValueBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public Builder clearArrayValue() {
+        if (arrayValueBuilder_ == null) {
+          arrayValue_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000080);
+          onChanged();
+        } else {
+          arrayValueBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public Builder removeArrayValue(int index) {
+        if (arrayValueBuilder_ == null) {
+          ensureArrayValueIsMutable();
+          arrayValue_.remove(index);
+          onChanged();
+        } else {
+          arrayValueBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue.Builder getArrayValueBuilder(
+          int index) {
+        return getArrayValueFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder getArrayValueOrBuilder(
+          int index) {
+        if (arrayValueBuilder_ == null) {
+          return arrayValue_.get(index);  } else {
+          return arrayValueBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public java.util.List<? extends org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+           getArrayValueOrBuilderList() {
+        if (arrayValueBuilder_ != null) {
+          return arrayValueBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(arrayValue_);
+        }
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue.Builder addArrayValueBuilder() {
+        return getArrayValueFieldBuilder().addBuilder(
+            org.apache.calcite.avatica.proto.Common.TypedValue.getDefaultInstance());
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.TypedValue.Builder addArrayValueBuilder(
+          int index) {
+        return getArrayValueFieldBuilder().addBuilder(
+            index, org.apache.calcite.avatica.proto.Common.TypedValue.getDefaultInstance());
+      }
+      /**
+       * <pre>
+       * The Array
+       * </pre>
+       *
+       * <code>repeated .TypedValue array_value = 8;</code>
+       */
+      public java.util.List<org.apache.calcite.avatica.proto.Common.TypedValue.Builder> 
+           getArrayValueBuilderList() {
+        return getArrayValueFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.calcite.avatica.proto.Common.TypedValue, org.apache.calcite.avatica.proto.Common.TypedValue.Builder, org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder> 
+          getArrayValueFieldBuilder() {
+        if (arrayValueBuilder_ == null) {
+          arrayValueBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3<
+              org.apache.calcite.avatica.proto.Common.TypedValue, org.apache.calcite.avatica.proto.Common.TypedValue.Builder, org.apache.calcite.avatica.proto.Common.TypedValueOrBuilder>(
+                  arrayValue_,
+                  ((bitField0_ & 0x00000080) == 0x00000080),
+                  getParentForChildren(),
+                  isClean());
+          arrayValue_ = null;
+        }
+        return arrayValueBuilder_;
+      }
+
+      private int componentType_ = 0;
+      /**
+       * <pre>
+       * If an Array, the representation for the array values
+       * </pre>
+       *
+       * <code>optional .Rep component_type = 9;</code>
+       */
+      public int getComponentTypeValue() {
+        return componentType_;
+      }
+      /**
+       * <pre>
+       * If an Array, the representation for the array values
+       * </pre>
+       *
+       * <code>optional .Rep component_type = 9;</code>
+       */
+      public Builder setComponentTypeValue(int value) {
+        componentType_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * If an Array, the representation for the array values
+       * </pre>
+       *
+       * <code>optional .Rep component_type = 9;</code>
+       */
+      public org.apache.calcite.avatica.proto.Common.Rep getComponentType() {
+        org.apache.calcite.avatica.proto.Common.Rep result = org.apache.calcite.avatica.proto.Common.Rep.valueOf(componentType_);
+        return result == null ? org.apache.calcite.avatica.proto.Common.Rep.UNRECOGNIZED : result;
+      }
+      /**
+       * <pre>
+       * If an Array, the representation for the array values
+       * </pre>
+       *
+       * <code>optional .Rep component_type = 9;</code>
+       */
+      public Builder setComponentType(org.apache.calcite.avatica.proto.Common.Rep value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        
+        componentType_ = value.getNumber();
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * If an Array, the representation for the array values
+       * </pre>
+       *
+       * <code>optional .Rep component_type = 9;</code>
+       */
+      public Builder clearComponentType() {
+        
+        componentType_ = 0;
+        onChanged();
+        return this;
+      }
       public final Builder setUnknownFields(
           final com.google.protobuf.UnknownFieldSet unknownFields) {
         return this;
@@ -18158,58 +18770,60 @@ public final class Common {
       " \001(\014\"\207\001\n\013ColumnValue\022\032\n\005value\030\001 \003(\0132\013.Ty",
       "pedValue\022 \n\013array_value\030\002 \003(\0132\013.TypedVal" +
       "ue\022\027\n\017has_array_value\030\003 \001(\010\022!\n\014scalar_va" +
-      "lue\030\004 \001(\0132\013.TypedValue\"\231\001\n\nTypedValue\022\022\n" +
+      "lue\030\004 \001(\0132\013.TypedValue\"\331\001\n\nTypedValue\022\022\n" +
       "\004type\030\001 \001(\0162\004.Rep\022\022\n\nbool_value\030\002 \001(\010\022\024\n" +
       "\014string_value\030\003 \001(\t\022\024\n\014number_value\030\004 \001(" +
       "\022\022\023\n\013bytes_value\030\005 \001(\014\022\024\n\014double_value\030\006" +
-      " \001(\001\022\014\n\004null\030\007 \001(\010\"\246\002\n\031MetaDataOperation" +
-      "Argument\022\024\n\014string_value\030\001 \001(\t\022\022\n\nbool_v" +
-      "alue\030\002 \001(\010\022\021\n\tint_value\030\003 \001(\021\022\033\n\023string_" +
-      "array_values\030\004 \003(\t\022\030\n\020int_array_values\030\005",
-      " \003(\021\0225\n\004type\030\006 \001(\0162\'.MetaDataOperationAr" +
-      "gument.ArgumentType\"^\n\014ArgumentType\022\n\n\006S" +
-      "TRING\020\000\022\010\n\004BOOL\020\001\022\007\n\003INT\020\002\022\023\n\017REPEATED_S" +
-      "TRING\020\003\022\020\n\014REPEATED_INT\020\004\022\010\n\004NULL\020\005\"\260\001\n\n" +
-      "QueryState\022\030\n\004type\030\001 \001(\0162\n.StateType\022\013\n\003" +
-      "sql\030\002 \001(\t\022\036\n\002op\030\003 \001(\0162\022.MetaDataOperatio" +
-      "n\022(\n\004args\030\004 \003(\0132\032.MetaDataOperationArgum" +
-      "ent\022\020\n\010has_args\030\005 \001(\010\022\017\n\007has_sql\030\006 \001(\010\022\016" +
-      "\n\006has_op\030\007 \001(\010*\237\001\n\rStatementType\022\n\n\006SELE" +
-      "CT\020\000\022\n\n\006INSERT\020\001\022\n\n\006UPDATE\020\002\022\n\n\006DELETE\020\003",
-      "\022\n\n\006UPSERT\020\004\022\t\n\005MERGE\020\005\022\r\n\tOTHER_DML\020\006\022\n" +
-      "\n\006CREATE\020\007\022\010\n\004DROP\020\010\022\t\n\005ALTER\020\t\022\r\n\tOTHER" +
-      "_DDL\020\n\022\010\n\004CALL\020\013*\342\003\n\003Rep\022\025\n\021PRIMITIVE_BO" +
-      "OLEAN\020\000\022\022\n\016PRIMITIVE_BYTE\020\001\022\022\n\016PRIMITIVE" +
-      "_CHAR\020\002\022\023\n\017PRIMITIVE_SHORT\020\003\022\021\n\rPRIMITIV" +
-      "E_INT\020\004\022\022\n\016PRIMITIVE_LONG\020\005\022\023\n\017PRIMITIVE" +
-      "_FLOAT\020\006\022\024\n\020PRIMITIVE_DOUBLE\020\007\022\013\n\007BOOLEA" +
-      "N\020\010\022\010\n\004BYTE\020\t\022\r\n\tCHARACTER\020\n\022\t\n\005SHORT\020\013\022" +
-      "\013\n\007INTEGER\020\014\022\010\n\004LONG\020\r\022\t\n\005FLOAT\020\016\022\n\n\006DOU" +
-      "BLE\020\017\022\017\n\013BIG_INTEGER\020\031\022\017\n\013BIG_DECIMAL\020\032\022",
-      "\021\n\rJAVA_SQL_TIME\020\020\022\026\n\022JAVA_SQL_TIMESTAMP" +
-      "\020\021\022\021\n\rJAVA_SQL_DATE\020\022\022\022\n\016JAVA_UTIL_DATE\020" +
-      "\023\022\017\n\013BYTE_STRING\020\024\022\n\n\006STRING\020\025\022\n\n\006NUMBER" +
-      "\020\026\022\n\n\006OBJECT\020\027\022\010\n\004NULL\020\030\022\t\n\005ARRAY\020\033\022\n\n\006S" +
-      "TRUCT\020\034\022\014\n\010MULTISET\020\035*^\n\010Severity\022\024\n\020UNK" +
-      "NOWN_SEVERITY\020\000\022\022\n\016FATAL_SEVERITY\020\001\022\022\n\016E" +
-      "RROR_SEVERITY\020\002\022\024\n\020WARNING_SEVERITY\020\003*\327\004" +
-      "\n\021MetaDataOperation\022\022\n\016GET_ATTRIBUTES\020\000\022" +
-      "\033\n\027GET_BEST_ROW_IDENTIFIER\020\001\022\020\n\014GET_CATA" +
-      "LOGS\020\002\022\036\n\032GET_CLIENT_INFO_PROPERTIES\020\003\022\031",
-      "\n\025GET_COLUMN_PRIVILEGES\020\004\022\017\n\013GET_COLUMNS" +
-      "\020\005\022\027\n\023GET_CROSS_REFERENCE\020\006\022\025\n\021GET_EXPOR" +
-      "TED_KEYS\020\007\022\030\n\024GET_FUNCTION_COLUMNS\020\010\022\021\n\r" +
-      "GET_FUNCTIONS\020\t\022\025\n\021GET_IMPORTED_KEYS\020\n\022\022" +
-      "\n\016GET_INDEX_INFO\020\013\022\024\n\020GET_PRIMARY_KEYS\020\014" +
-      "\022\031\n\025GET_PROCEDURE_COLUMNS\020\r\022\022\n\016GET_PROCE" +
-      "DURES\020\016\022\026\n\022GET_PSEUDO_COLUMNS\020\017\022\017\n\013GET_S" +
-      "CHEMAS\020\020\022\031\n\025GET_SCHEMAS_WITH_ARGS\020\021\022\024\n\020G" +
-      "ET_SUPER_TABLES\020\022\022\023\n\017GET_SUPER_TYPES\020\023\022\030" +
-      "\n\024GET_TABLE_PRIVILEGES\020\024\022\016\n\nGET_TABLES\020\025",
-      "\022\023\n\017GET_TABLE_TYPES\020\026\022\021\n\rGET_TYPE_INFO\020\027" +
-      "\022\014\n\010GET_UDTS\020\030\022\027\n\023GET_VERSION_COLUMNS\020\031*" +
-      "\"\n\tStateType\022\007\n\003SQL\020\000\022\014\n\010METADATA\020\001B\"\n o" +
-      "rg.apache.calcite.avatica.protob\006proto3"
+      " \001(\001\022\014\n\004null\030\007 \001(\010\022 \n\013array_value\030\010 \003(\0132" +
+      "\013.TypedValue\022\034\n\016component_type\030\t \001(\0162\004.R" +
+      "ep\"\246\002\n\031MetaDataOperationArgument\022\024\n\014stri" +
+      "ng_value\030\001 \001(\t\022\022\n\nbool_value\030\002 \001(\010\022\021\n\tin",
+      "t_value\030\003 \001(\021\022\033\n\023string_array_values\030\004 \003" +
+      "(\t\022\030\n\020int_array_values\030\005 \003(\021\0225\n\004type\030\006 \001" +
+      "(\0162\'.MetaDataOperationArgument.ArgumentT" +
+      "ype\"^\n\014ArgumentType\022\n\n\006STRING\020\000\022\010\n\004BOOL\020" +
+      "\001\022\007\n\003INT\020\002\022\023\n\017REPEATED_STRING\020\003\022\020\n\014REPEA" +
+      "TED_INT\020\004\022\010\n\004NULL\020\005\"\260\001\n\nQueryState\022\030\n\004ty" +
+      "pe\030\001 \001(\0162\n.StateType\022\013\n\003sql\030\002 \001(\t\022\036\n\002op\030" +
+      "\003 \001(\0162\022.MetaDataOperation\022(\n\004args\030\004 \003(\0132" +
+      "\032.MetaDataOperationArgument\022\020\n\010has_args\030" +
+      "\005 \001(\010\022\017\n\007has_sql\030\006 \001(\010\022\016\n\006has_op\030\007 \001(\010*\237",
+      "\001\n\rStatementType\022\n\n\006SELECT\020\000\022\n\n\006INSERT\020\001" +
+      "\022\n\n\006UPDATE\020\002\022\n\n\006DELETE\020\003\022\n\n\006UPSERT\020\004\022\t\n\005" +
+      "MERGE\020\005\022\r\n\tOTHER_DML\020\006\022\n\n\006CREATE\020\007\022\010\n\004DR" +
+      "OP\020\010\022\t\n\005ALTER\020\t\022\r\n\tOTHER_DDL\020\n\022\010\n\004CALL\020\013" +
+      "*\342\003\n\003Rep\022\025\n\021PRIMITIVE_BOOLEAN\020\000\022\022\n\016PRIMI" +
+      "TIVE_BYTE\020\001\022\022\n\016PRIMITIVE_CHAR\020\002\022\023\n\017PRIMI" +
+      "TIVE_SHORT\020\003\022\021\n\rPRIMITIVE_INT\020\004\022\022\n\016PRIMI" +
+      "TIVE_LONG\020\005\022\023\n\017PRIMITIVE_FLOAT\020\006\022\024\n\020PRIM" +
+      "ITIVE_DOUBLE\020\007\022\013\n\007BOOLEAN\020\010\022\010\n\004BYTE\020\t\022\r\n" +
+      "\tCHARACTER\020\n\022\t\n\005SHORT\020\013\022\013\n\007INTEGER\020\014\022\010\n\004",
+      "LONG\020\r\022\t\n\005FLOAT\020\016\022\n\n\006DOUBLE\020\017\022\017\n\013BIG_INT" +
+      "EGER\020\031\022\017\n\013BIG_DECIMAL\020\032\022\021\n\rJAVA_SQL_TIME" +
+      "\020\020\022\026\n\022JAVA_SQL_TIMESTAMP\020\021\022\021\n\rJAVA_SQL_D" +
+      "ATE\020\022\022\022\n\016JAVA_UTIL_DATE\020\023\022\017\n\013BYTE_STRING" +
+      "\020\024\022\n\n\006STRING\020\025\022\n\n\006NUMBER\020\026\022\n\n\006OBJECT\020\027\022\010" +
+      "\n\004NULL\020\030\022\t\n\005ARRAY\020\033\022\n\n\006STRUCT\020\034\022\014\n\010MULTI" +
+      "SET\020\035*^\n\010Severity\022\024\n\020UNKNOWN_SEVERITY\020\000\022" +
+      "\022\n\016FATAL_SEVERITY\020\001\022\022\n\016ERROR_SEVERITY\020\002\022" +
+      "\024\n\020WARNING_SEVERITY\020\003*\327\004\n\021MetaDataOperat" +
+      "ion\022\022\n\016GET_ATTRIBUTES\020\000\022\033\n\027GET_BEST_ROW_",
+      "IDENTIFIER\020\001\022\020\n\014GET_CATALOGS\020\002\022\036\n\032GET_CL" +
+      "IENT_INFO_PROPERTIES\020\003\022\031\n\025GET_COLUMN_PRI" +
+      "VILEGES\020\004\022\017\n\013GET_COLUMNS\020\005\022\027\n\023GET_CROSS_" +
+      "REFERENCE\020\006\022\025\n\021GET_EXPORTED_KEYS\020\007\022\030\n\024GE" +
+      "T_FUNCTION_COLUMNS\020\010\022\021\n\rGET_FUNCTIONS\020\t\022" +
+      "\025\n\021GET_IMPORTED_KEYS\020\n\022\022\n\016GET_INDEX_INFO" +
+      "\020\013\022\024\n\020GET_PRIMARY_KEYS\020\014\022\031\n\025GET_PROCEDUR" +
+      "E_COLUMNS\020\r\022\022\n\016GET_PROCEDURES\020\016\022\026\n\022GET_P" +
+      "SEUDO_COLUMNS\020\017\022\017\n\013GET_SCHEMAS\020\020\022\031\n\025GET_" +
+      "SCHEMAS_WITH_ARGS\020\021\022\024\n\020GET_SUPER_TABLES\020",
+      "\022\022\023\n\017GET_SUPER_TYPES\020\023\022\030\n\024GET_TABLE_PRIV" +
+      "ILEGES\020\024\022\016\n\nGET_TABLES\020\025\022\023\n\017GET_TABLE_TY" +
+      "PES\020\026\022\021\n\rGET_TYPE_INFO\020\027\022\014\n\010GET_UDTS\020\030\022\027" +
+      "\n\023GET_VERSION_COLUMNS\020\031*\"\n\tStateType\022\007\n\003" +
+      "SQL\020\000\022\014\n\010METADATA\020\001B\"\n org.apache.calcit" +
+      "e.avatica.protob\006proto3"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -18300,7 +18914,7 @@ public final class Common {
     internal_static_TypedValue_fieldAccessorTable = new
       com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_TypedValue_descriptor,
-        new java.lang.String[] { "Type", "BoolValue", "StringValue", "NumberValue", "BytesValue", "DoubleValue", "Null", });
+        new java.lang.String[] { "Type", "BoolValue", "StringValue", "NumberValue", "BytesValue", "DoubleValue", "Null", "ArrayValue", "ComponentType", });
     internal_static_MetaDataOperationArgument_descriptor =
       getDescriptor().getMessageTypes().get(13);
     internal_static_MetaDataOperationArgument_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/remote/Service.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/Service.java b/core/src/main/java/org/apache/calcite/avatica/remote/Service.java
index bcbc557..8178ecd 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/Service.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/Service.java
@@ -1045,7 +1045,11 @@ public interface Service {
       if (msg.getHasParameterValues()) {
         values = new ArrayList<>(msg.getParameterValuesCount());
         for (Common.TypedValue valueProto : msg.getParameterValuesList()) {
-          values.add(TypedValue.fromProto(valueProto));
+          if (TypedValue.NULL_PROTO.equals(valueProto)) {
+            values.add(null);
+          } else {
+            values.add(TypedValue.fromProto(valueProto));
+          }
         }
       }
 
@@ -1072,7 +1076,7 @@ public interface Service {
         builder.setHasParameterValues(true);
         for (TypedValue paramValue : parameterValues) {
           if (paramValue == null) {
-            builder.addParameterValues(TypedValue.NULL.toProto());
+            builder.addParameterValues(TypedValue.NULL_PROTO);
           } else {
             builder.addParameterValues(paramValue.toProto());
           }

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java b/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
index 7f1c752..04268bb 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
@@ -16,9 +16,13 @@
  */
 package org.apache.calcite.avatica.remote;
 
+import org.apache.calcite.avatica.AvaticaUtils;
 import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.ColumnMetaData.AvaticaType;
 import org.apache.calcite.avatica.ColumnMetaData.Rep;
+import org.apache.calcite.avatica.SqlType;
 import org.apache.calcite.avatica.proto.Common;
+import org.apache.calcite.avatica.util.ArrayFactoryImpl;
 import org.apache.calcite.avatica.util.Base64;
 import org.apache.calcite.avatica.util.ByteString;
 import org.apache.calcite.avatica.util.DateTimeUtils;
@@ -30,6 +34,11 @@ import com.google.protobuf.UnsafeByteOperations;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
+import java.sql.Array;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.Date;
@@ -101,6 +110,10 @@ import static java.nio.charset.StandardCharsets.UTF_8;
  *     <td>DECIMAL</td>
  *                   <td>BigDecimal</td> <td>Number</td> <td>BigDecimal</td>
  *   </tr>
+ *   <tr>
+ *     <td>ARRAY</td>
+ *                  <td>Array</td> <td>List&lt;Object&gt;</td> <td>List&lt;Object&gt;</td>
+ *   </tr>
  * </table>
  *
  * Note:
@@ -134,6 +147,7 @@ public class TypedValue {
 
   public static final TypedValue NULL =
       new TypedValue(ColumnMetaData.Rep.OBJECT, null);
+  public static final Common.TypedValue NULL_PROTO = NULL.toProto();
 
   /** Type of the value. */
   public final ColumnMetaData.Rep type;
@@ -144,8 +158,16 @@ public class TypedValue {
    * For example, byte arrays are represented as String. */
   public final Object value;
 
+  /** Non-null for ARRAYs, the type of the values stored in the ARRAY. Null for all other cases. */
+  public final ColumnMetaData.Rep componentType;
+
   private TypedValue(ColumnMetaData.Rep rep, Object value) {
+    this(rep, null, value);
+  }
+
+  private TypedValue(ColumnMetaData.Rep rep, ColumnMetaData.Rep componentType, Object value) {
     this.type = rep;
+    this.componentType = componentType;
     this.value = value;
     assert isSerial(rep, value) : "rep: " + rep + ", value: " + value;
   }
@@ -194,7 +216,27 @@ public class TypedValue {
     if (value == null) {
       return NULL;
     }
-    return new TypedValue(rep, jdbcToSerial(rep, value, calendar));
+    final Object serialValue;
+    if (ColumnMetaData.Rep.ARRAY == rep) {
+      // Sanity check that we were given an Array
+      if (null != value && !(value instanceof Array)) {
+        throw new IllegalArgumentException("Provided Rep was ARRAY, but the value was "
+            + value.getClass());
+      }
+      final Array array = (Array) value;
+      try {
+        SqlType type = SqlType.valueOf(array.getBaseType());
+        serialValue = jdbcToSerial(rep, array, calendar, type);
+        // Because an Array may have null entries, we must always return the non-primitive type
+        // variants of the array values.
+        return new TypedValue(rep, Rep.nonPrimitiveRepOf(type), serialValue);
+      } catch (SQLException e) {
+        throw new RuntimeException("Could not extract Array component type", e);
+      }
+    } else {
+      serialValue = jdbcToSerial(rep, value, calendar);
+    }
+    return new TypedValue(rep, serialValue);
   }
 
   /** Creates a TypedValue from a value in JDBC representation,
@@ -251,6 +293,9 @@ public class TypedValue {
           : new BigDecimal(((Number) value).longValue());
     case BYTE_STRING:
       return ByteString.ofBase64((String) value);
+    case ARRAY:
+      //List<Object>
+      return value;
     default:
       throw new IllegalArgumentException("cannot convert " + value + " ("
           + value.getClass() + ") to " + rep);
@@ -266,7 +311,7 @@ public class TypedValue {
     if (value == null) {
       return null;
     }
-    return serialToJdbc(type, value, calendar);
+    return serialToJdbc(type, componentType, value, calendar);
   }
 
   /**
@@ -277,7 +322,8 @@ public class TypedValue {
    * @param calendar A calendar instance
    * @return The JDBC representation of the value.
    */
-  private static Object serialToJdbc(ColumnMetaData.Rep type, Object value, Calendar calendar) {
+  private static Object serialToJdbc(ColumnMetaData.Rep type, ColumnMetaData.Rep componentRep,
+      Object value, Calendar calendar) {
     switch (type) {
     case BYTE_STRING:
       return ByteString.ofBase64((String) value).getBytes();
@@ -291,6 +337,27 @@ public class TypedValue {
       return new java.sql.Time(adjust((Number) value, calendar));
     case JAVA_SQL_TIMESTAMP:
       return new java.sql.Timestamp(adjust((Number) value, calendar));
+    case ARRAY:
+      if (null == value) {
+        return null;
+      }
+      final List<?> list = (List<?>) value;
+      final List<Object> copy = new ArrayList<>(list.size());
+      // Copy the list from the serial representation to a JDBC representation
+      for (Object o : list) {
+        if (null == o) {
+          copy.add(null);
+        } else if (o instanceof TypedValue) {
+          // Protobuf can maintain the TypedValue hierarchy to simplify things
+          copy.add(((TypedValue) o).toJdbc(calendar));
+        } else {
+          // We can't get the above recursion with the JSON serialization
+          copy.add(serialToJdbc(componentRep, null, o, calendar));
+        }
+      }
+      AvaticaType elementType = new AvaticaType(componentRep.typeId, componentRep.name(),
+          componentRep);
+      return new ArrayFactoryImpl(calendar.getTimeZone()).createArray(elementType, copy);
     default:
       return serialToLocal(type, value);
     }
@@ -304,10 +371,18 @@ public class TypedValue {
     return t;
   }
 
+  private static Object jdbcToSerial(ColumnMetaData.Rep rep, Object value,
+      Calendar calendar) {
+    return jdbcToSerial(rep, value, calendar, null);
+  }
+
   /** Converts a value from JDBC format to a type that can be serialized as
    * JSON. */
   private static Object jdbcToSerial(ColumnMetaData.Rep rep, Object value,
-      Calendar calendar) {
+      Calendar calendar, SqlType componentType) {
+    if (null == value) {
+      return null;
+    }
     switch (rep) {
     case BYTE_STRING:
       return new ByteString((byte[]) value).toBase64String();
@@ -327,6 +402,58 @@ public class TypedValue {
       default:
         return t;
       }
+    case ARRAY:
+      Array array = (Array) value;
+      Objects.requireNonNull(componentType, "Component Type must not be null for ARRAYs");
+      try {
+        switch (componentType) {
+        case BINARY:
+        case VARBINARY:
+        case LONGVARBINARY:
+          Object[] byteStrings = (Object[]) array.getArray();
+          List<String> convertedStrings = new ArrayList<>(byteStrings.length);
+          for (Object byteString : byteStrings) {
+            convertedStrings.add(
+                (String) jdbcToSerial(Rep.BYTE_STRING, byteString, calendar, null));
+          }
+          return convertedStrings;
+        case DATE:
+        case TIME:
+          Object[] dates = (Object[]) array.getArray();
+          List<Integer> serializedDates = new ArrayList<>(dates.length);
+          for (Object obj : dates) {
+            Date date = (Date) obj;
+            if (null == obj) {
+              serializedDates.add(null);
+            } else if (componentType == SqlType.DATE) {
+              serializedDates.add((int) jdbcToSerial(Rep.JAVA_SQL_DATE, date, calendar, null));
+            } else if (componentType == SqlType.TIME) {
+              serializedDates.add((int) jdbcToSerial(Rep.JAVA_SQL_TIME, date, calendar, null));
+            } else {
+              throw new RuntimeException("Unexpected type: " + componentType);
+            }
+          }
+          return serializedDates;
+        case TIMESTAMP:
+          Object[] timestamps = (Object[]) array.getArray();
+          List<Long> serializedTimestamps = new ArrayList<>(timestamps.length);
+          for (Object obj : timestamps) {
+            Timestamp timestamp = (Timestamp) obj;
+            if (null == obj) {
+              serializedTimestamps.add(null);
+            } else {
+              serializedTimestamps.add(
+                  (long) jdbcToSerial(Rep.JAVA_SQL_TIMESTAMP, timestamp, calendar, null));
+            }
+          }
+          return serializedTimestamps;
+        default:
+          // Either a primitive array or Object[], converted into List<Object>
+          return AvaticaUtils.primitiveList(array.getArray());
+        }
+      } catch (SQLException e) {
+        throw new RuntimeException("Could not obtain base array object", e);
+      }
     default:
       return value;
     }
@@ -363,6 +490,17 @@ public class TypedValue {
     // Protobuf has an explicit BIG_DECIMAL representation enum value.
     if (Common.Rep.NUMBER == protoRep && value instanceof BigDecimal) {
       protoRep = Common.Rep.BIG_DECIMAL;
+    } else if (Common.Rep.ARRAY == protoRep) {
+      // This column value is an Array (many TypedValue's)
+      builder.setType(Common.Rep.ARRAY);
+      // Get the array component's type
+      Common.Rep protoComponentRep = componentType.toProto();
+      // Set the array's component on the message
+      builder.setComponentType(protoComponentRep);
+      // Serialize that array into the builder
+      @SuppressWarnings("unchecked")
+      List<Object> list = (List<Object>) value;
+      return serializeArray(list, builder, protoComponentRep);
     }
 
     // Serialize the type into the protobuf
@@ -371,6 +509,33 @@ public class TypedValue {
     return builder.build();
   }
 
+  Common.TypedValue serializeArray(List<Object> list, Common.TypedValue.Builder builder,
+      Common.Rep protoArrayComponentRep) {
+    for (Object element : list) {
+      if (element instanceof List) {
+        // We have a list of lists: recursively build up the protobuf
+        @SuppressWarnings("unchecked")
+        List<Object> subList = (List<Object>) element;
+        Common.TypedValue.Builder subListBuilder = Common.TypedValue.newBuilder();
+        // This is "technically" an array, but we just persist the underlying component type
+        subListBuilder.setType(protoArrayComponentRep);
+        Common.TypedValue protoSubList = serializeArray(subList, subListBuilder,
+            protoArrayComponentRep);
+        builder.addArrayValue(protoSubList);
+      } else {
+        // We have a list of "scalars", just serialize the value
+        Common.TypedValue.Builder elementBuilder = Common.TypedValue.newBuilder();
+        if (null == element) {
+          writeToProtoWithType(elementBuilder, null, Common.Rep.NULL);
+        } else {
+          writeToProtoWithType(elementBuilder, element, protoArrayComponentRep);
+        }
+        builder.addArrayValue(elementBuilder.build());
+      }
+    }
+    return builder.build();
+  }
+
   private static void writeToProtoWithType(Common.TypedValue.Builder builder, Object o,
       Common.Rep type) {
     builder.setType(type);
@@ -429,13 +594,31 @@ public class TypedValue {
       return;
     case JAVA_SQL_DATE:
     case JAVA_SQL_TIME:
-      // Persisted as integers
-      builder.setNumberValue(Integer.valueOf((int) o).longValue());
+      long sqlDateOrTime;
+      if (o instanceof java.sql.Date) {
+        sqlDateOrTime = ((java.sql.Date) o).getTime();
+      } else if (o instanceof java.sql.Time) {
+        sqlDateOrTime = ((java.sql.Time) o).getTime();
+      } else if (o instanceof Integer) {
+        sqlDateOrTime = ((Integer) o).longValue();
+      } else {
+        sqlDateOrTime = (long) o;
+      }
+      // Persisted as numbers
+      builder.setNumberValue(sqlDateOrTime);
       return;
     case JAVA_SQL_TIMESTAMP:
     case JAVA_UTIL_DATE:
+      long sqlTimestampOrUtilDate;
+      if (o instanceof java.sql.Timestamp) {
+        sqlTimestampOrUtilDate = ((java.sql.Timestamp) o).getTime();
+      } else if (o instanceof java.util.Date) {
+        sqlTimestampOrUtilDate = ((java.util.Date) o).getTime();
+      } else {
+        sqlTimestampOrUtilDate = (long) o;
+      }
       // Persisted as longs
-      builder.setNumberValue((long) o);
+      builder.setNumberValue(sqlTimestampOrUtilDate);
       return;
     case BIG_INTEGER:
       byte[] byteRep = ((BigInteger) o).toByteArray();
@@ -476,9 +659,10 @@ public class TypedValue {
    */
   public static TypedValue fromProto(Common.TypedValue proto) {
     ColumnMetaData.Rep rep = ColumnMetaData.Rep.fromProto(proto.getType());
+    ColumnMetaData.Rep componentRep = ColumnMetaData.Rep.fromProto(proto.getComponentType());
     Object value = getSerialFromProto(proto);
 
-    return new TypedValue(rep, value);
+    return new TypedValue(rep, componentRep, value);
   }
 
   /**
@@ -547,6 +731,16 @@ public class TypedValue {
       return Long.valueOf(protoValue.getNumberValue());
     case NULL:
       return null;
+    case ARRAY:
+      final List<Common.TypedValue> protoList = protoValue.getArrayValueList();
+      final List<Object> list = new ArrayList<>(protoList.size());
+      for (Common.TypedValue protoElement : protoList) {
+        // Deserialize the TypedValue protobuf into the JDBC type
+        TypedValue listElement = TypedValue.fromProto(protoElement);
+        // Must preserve the TypedValue so serial/jdbc/local conversion can work as expected
+        list.add(listElement);
+      }
+      return list;
     case OBJECT:
       if (protoValue.getNull()) {
         return null;
@@ -569,35 +763,97 @@ public class TypedValue {
    * @param builder The TypedValue protobuf builder
    * @param o The object (value)
    */
-  public static void toProto(Common.TypedValue.Builder builder, Object o) {
+  public static Common.Rep toProto(Common.TypedValue.Builder builder, Object o) {
     // Numbers
     if (o instanceof Byte) {
       writeToProtoWithType(builder, o, Common.Rep.BYTE);
+      return Common.Rep.BYTE;
     } else if (o instanceof Short) {
       writeToProtoWithType(builder, o, Common.Rep.SHORT);
+      return Common.Rep.SHORT;
     } else if (o instanceof Integer) {
       writeToProtoWithType(builder, o, Common.Rep.INTEGER);
+      return Common.Rep.INTEGER;
     } else if (o instanceof Long) {
       writeToProtoWithType(builder, o, Common.Rep.LONG);
+      return Common.Rep.LONG;
     } else if (o instanceof Double) {
       writeToProtoWithType(builder, o, Common.Rep.DOUBLE);
+      return Common.Rep.DOUBLE;
     } else if (o instanceof Float) {
       writeToProtoWithType(builder, ((Float) o).longValue(), Common.Rep.FLOAT);
+      return Common.Rep.FLOAT;
     } else if (o instanceof BigDecimal) {
       writeToProtoWithType(builder, o, Common.Rep.BIG_DECIMAL);
+      return Common.Rep.BIG_DECIMAL;
     // Strings
     } else if (o instanceof String) {
       writeToProtoWithType(builder, o, Common.Rep.STRING);
+      return Common.Rep.STRING;
     } else if (o instanceof Character) {
       writeToProtoWithType(builder, o.toString(), Common.Rep.CHARACTER);
+      return Common.Rep.CHARACTER;
     // Bytes
     } else if (o instanceof byte[]) {
       writeToProtoWithType(builder, o, Common.Rep.BYTE_STRING);
+      return Common.Rep.BYTE_STRING;
     // Boolean
     } else if (o instanceof Boolean) {
       writeToProtoWithType(builder, o, Common.Rep.BOOLEAN);
+      return Common.Rep.BOOLEAN;
+    } else if (o instanceof Timestamp) {
+      writeToProtoWithType(builder, o, Common.Rep.JAVA_SQL_TIMESTAMP);
+      return Common.Rep.JAVA_SQL_TIMESTAMP;
+    } else if (o instanceof Date) {
+      writeToProtoWithType(builder, o, Common.Rep.JAVA_SQL_DATE);
+      return Common.Rep.JAVA_SQL_DATE;
+    } else if (o instanceof Time) {
+      writeToProtoWithType(builder, o, Common.Rep.JAVA_SQL_TIME);
+      return Common.Rep.JAVA_SQL_TIME;
+    } else if (o instanceof List) {
+      // Treat a List as an Array
+      builder.setType(Common.Rep.ARRAY);
+      builder.setComponentType(Common.Rep.OBJECT);
+      boolean setComponentType = false;
+      for (Object listElement : (List<?>) o) {
+        Common.TypedValue.Builder listElementBuilder = Common.TypedValue.newBuilder();
+        // Recurse on each list element
+        Common.Rep componentRep = toProto(listElementBuilder, listElement);
+        if (!setComponentType) {
+          if (Common.Rep.NULL != componentRep) {
+            builder.setComponentType(componentRep);
+          }
+          setComponentType = true;
+        }
+        builder.addArrayValue(listElementBuilder.build());
+      }
+      return Common.Rep.ARRAY;
+    } else if (o instanceof Array) {
+      builder.setType(Common.Rep.ARRAY);
+      Array a = (Array) o;
+      try {
+        ResultSet rs = a.getResultSet();
+        builder.setComponentType(Common.Rep.OBJECT);
+        boolean setComponentType = false;
+        while (rs.next()) {
+          Common.TypedValue.Builder listElementBuilder = Common.TypedValue.newBuilder();
+          Object arrayValue = rs.getObject(2);
+          Common.Rep componentRep = toProto(listElementBuilder, arrayValue);
+          if (!setComponentType) {
+            if (Common.Rep.NULL != componentRep) {
+              builder.setComponentType(componentRep);
+            }
+            setComponentType = true;
+          }
+          builder.addArrayValue(listElementBuilder.build());
+        }
+      } catch (SQLException e) {
+        throw new RuntimeException("Could not serialize ARRAY", e);
+      }
+      return Common.Rep.ARRAY;
     } else if (null == o) {
       writeToProtoWithType(builder, o, Common.Rep.NULL);
+      return Common.Rep.NULL;
     // Unhandled
     } else {
       throw new RuntimeException("Unhandled type in Frame: " + o.getClass());
@@ -617,8 +873,7 @@ public class TypedValue {
     if (null == o) {
       return o;
     }
-    return serialToJdbc(Rep.fromProto(protoValue.getType()), o, calendar);
-    //return protoSerialToJdbc(protoValue.getType(), o, Objects.requireNonNull(calendar));
+    return serialToJdbc(Rep.fromProto(protoValue.getType()), null, o, calendar);
   }
 
   @Override public int hashCode() {


[2/4] calcite-avatica git commit: [CALCITE-1050] Array support for Avatica

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java b/core/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
index 77739ab..ba6a669 100644
--- a/core/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
+++ b/core/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java
@@ -177,9 +177,9 @@ public abstract class AbstractCursor implements Cursor {
           (ColumnMetaData.ArrayType) columnMetaData.type;
       final SlotGetter componentGetter = new SlotGetter();
       final Accessor componentAccessor =
-          createAccessor(ColumnMetaData.dummy(arrayType.component, true),
+          createAccessor(ColumnMetaData.dummy(arrayType.getComponent(), true),
               componentGetter, localCalendar, factory);
-      return new ArrayAccessor(getter, arrayType.component, componentAccessor,
+      return new ArrayAccessor(getter, arrayType.getComponent(), componentAccessor,
           componentGetter, factory);
     case Types.STRUCT:
       switch (columnMetaData.type.rep) {
@@ -201,14 +201,14 @@ public abstract class AbstractCursor implements Cursor {
       }
     case Types.JAVA_OBJECT:
     case Types.OTHER: // e.g. map
-      if (columnMetaData.type.name.startsWith("INTERVAL_")) {
-        int end = columnMetaData.type.name.indexOf("(");
+      if (columnMetaData.type.getName().startsWith("INTERVAL_")) {
+        int end = columnMetaData.type.getName().indexOf("(");
         if (end < 0) {
-          end = columnMetaData.type.name.length();
+          end = columnMetaData.type.getName().length();
         }
         TimeUnitRange range =
             TimeUnitRange.valueOf(
-                columnMetaData.type.name.substring("INTERVAL_".length(), end));
+                columnMetaData.type.getName().substring("INTERVAL_".length(), end));
         if (range.monthly()) {
           return new IntervalYearMonthAccessor(getter, range);
         } else {
@@ -480,8 +480,13 @@ public abstract class AbstractCursor implements Cursor {
     }
 
     public byte getByte() throws SQLException {
-      Byte o = (Byte) getObject();
-      return o == null ? 0 : o;
+      Object obj = getObject();
+      if (null == obj) {
+        return 0;
+      } else if (obj instanceof Integer) {
+        return ((Integer) obj).byteValue();
+      }
+      return (Byte) obj;
     }
 
     public long getLong() throws SQLException {
@@ -499,8 +504,13 @@ public abstract class AbstractCursor implements Cursor {
     }
 
     public short getShort() throws SQLException {
-      Short o = (Short) getObject();
-      return o == null ? 0 : o;
+      Object obj = getObject();
+      if (null == obj) {
+        return 0;
+      } else if (obj instanceof Integer) {
+        return ((Integer) obj).shortValue();
+      }
+      return (Short) obj;
     }
 
     public long getLong() throws SQLException {
@@ -603,8 +613,13 @@ public abstract class AbstractCursor implements Cursor {
     }
 
     public double getDouble() throws SQLException {
-      Double o = (Double) getObject();
-      return o == null ? 0d : o;
+      Object obj = getObject();
+      if (null == obj) {
+        return 0d;
+      } else if (obj instanceof BigDecimal) {
+        return ((BigDecimal) obj).doubleValue();
+      }
+      return (Double) obj;
     }
   }
 
@@ -725,7 +740,11 @@ public abstract class AbstractCursor implements Cursor {
     }
 
     public String getString() throws SQLException {
-      return (String) getObject();
+      final Object obj = getObject();
+      if (obj instanceof String) {
+        return (String) obj;
+      }
+      return null == obj ? null : obj.toString();
     }
 
     @Override public byte[] getBytes() throws SQLException {
@@ -792,8 +811,10 @@ public abstract class AbstractCursor implements Cursor {
       if (obj instanceof ByteString) {
         return ((ByteString) obj).getBytes();
       } else if (obj instanceof String) {
-        return ((String) obj).getBytes(StandardCharsets.UTF_8);
+        // Need to unwind the base64 for JSON
+        return ByteString.parseBase64((String) obj);
       } else if (obj instanceof byte[]) {
+        // Protobuf would have a byte array
         return (byte[]) obj;
       } else {
         throw new RuntimeException("Cannot handle " + obj.getClass() + " as bytes");
@@ -1235,7 +1256,7 @@ public abstract class AbstractCursor implements Cursor {
    * Accessor that assumes that the underlying value is an ARRAY;
    * corresponds to {@link java.sql.Types#ARRAY}.
    */
-  static class ArrayAccessor extends AccessorImpl {
+  public static class ArrayAccessor extends AccessorImpl {
     final ColumnMetaData.AvaticaType componentType;
     final Accessor componentAccessor;
     final SlotGetter componentSlotGetter;
@@ -1253,20 +1274,80 @@ public abstract class AbstractCursor implements Cursor {
 
     @Override public Object getObject() throws SQLException {
       final Object object = super.getObject();
-      if (object == null || object instanceof List) {
+      if (object == null || object instanceof ArrayImpl) {
         return object;
+      } else if (object instanceof List) {
+        List<?> list = (List<?>) object;
+        // Run the array values through the component accessor
+        List<Object> convertedValues = new ArrayList<>(list.size());
+        for (Object val : list) {
+          if (null == val) {
+            convertedValues.add(null);
+          } else {
+            // Set the current value in the SlotGetter so we can use the Accessor to coerce it.
+            componentSlotGetter.slot = val;
+            convertedValues.add(convertValue());
+          }
+        }
+        return convertedValues;
       }
-      // The object can be java array in case of user-provided class for row
-      // storage.
+      // The object can be java array in case of user-provided class for row storage.
       return AvaticaUtils.primitiveList(object);
     }
 
-    @Override public Array getArray() throws SQLException {
-      final List list = (List) getObject();
-      if (list == null) {
+    private Object convertValue() throws SQLException {
+      switch (componentType.id) {
+      case Types.BOOLEAN:
+      case Types.BIT:
+        return componentAccessor.getBoolean();
+      case Types.TINYINT:
+        return componentAccessor.getByte();
+      case Types.SMALLINT:
+        return componentAccessor.getShort();
+      case Types.INTEGER:
+        return componentAccessor.getInt();
+      case Types.BIGINT:
+        return componentAccessor.getLong();
+      case Types.FLOAT:
+        return componentAccessor.getFloat();
+      case Types.DOUBLE:
+        return componentAccessor.getDouble();
+      case Types.ARRAY:
+        return componentAccessor.getArray();
+      case Types.CHAR:
+      case Types.VARCHAR:
+      case Types.LONGVARCHAR:
+      case Types.NCHAR:
+      case Types.LONGNVARCHAR:
+        return componentAccessor.getString();
+      case Types.BINARY:
+      case Types.VARBINARY:
+      case Types.LONGVARBINARY:
+        return componentAccessor.getBytes();
+      case Types.DECIMAL:
+        return componentAccessor.getBigDecimal();
+      case Types.DATE:
+      case Types.TIME:
+      case Types.TIMESTAMP:
+      case Types.STRUCT:
+      case Types.JAVA_OBJECT:
+        return componentAccessor.getObject();
+      default:
+        throw new IllegalStateException("Unhandled ARRAY component type: " + componentType.rep
+            + ", id: " + componentType.id);
+      }
+    }
+
+    @SuppressWarnings("unchecked") @Override public Array getArray() throws SQLException {
+      final Object o = getObject();
+      if (o == null) {
         return null;
       }
-      return new ArrayImpl(list, this);
+      if (o instanceof ArrayImpl) {
+        return (ArrayImpl) o;
+      }
+      // If it's not an Array already, assume it is a List.
+      return new ArrayImpl((List<Object>) o, this);
     }
 
     @Override public String getString() throws SQLException {
@@ -1291,10 +1372,22 @@ public abstract class AbstractCursor implements Cursor {
       return getStruct();
     }
 
+    @SuppressWarnings("unchecked")
+    @Override public <T> T getObject(Class<T> clz) throws SQLException {
+      // getStruct() is not exposed on Accessor, only AccessorImpl. getObject(Class) is exposed,
+      // so we can make it do the right thing (call getStruct()).
+      if (clz.equals(Struct.class)) {
+        return (T) getStruct();
+      }
+      return super.getObject(clz);
+    }
+
     @Override public Struct getStruct() throws SQLException {
       final Object o = super.getObject();
       if (o == null) {
         return null;
+      } else if (o instanceof StructImpl) {
+        return (StructImpl) o;
       } else if (o instanceof List) {
         return new StructImpl((List) o);
       } else {

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/util/ArrayFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/util/ArrayFactoryImpl.java b/core/src/main/java/org/apache/calcite/avatica/util/ArrayFactoryImpl.java
new file mode 100644
index 0000000..c90e999
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/avatica/util/ArrayFactoryImpl.java
@@ -0,0 +1,142 @@
+/*
+ * 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.AvaticaParameter;
+import org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.calcite.avatica.AvaticaResultSetMetaData;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.ColumnMetaData.ArrayType;
+import org.apache.calcite.avatica.ColumnMetaData.AvaticaType;
+import org.apache.calcite.avatica.ColumnMetaData.Rep;
+import org.apache.calcite.avatica.ColumnMetaData.ScalarType;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.QueryState;
+import org.apache.calcite.avatica.util.AbstractCursor.ArrayAccessor;
+import org.apache.calcite.avatica.util.Cursor.Accessor;
+
+import java.sql.Array;
+import java.sql.ResultSet;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.TimeZone;
+
+/**
+ * Implementation of {@link ArrayImpl.Factory}.
+ */
+public class ArrayFactoryImpl implements ArrayImpl.Factory {
+  private TimeZone timeZone;
+
+  public ArrayFactoryImpl(TimeZone timeZone) {
+    this.timeZone = Objects.requireNonNull(timeZone);
+  }
+
+  @Override public ResultSet create(AvaticaType elementType, Iterable<Object> elements) {
+    // The ColumnMetaData for offset "1" in the ResultSet for an Array.
+    ScalarType arrayOffsetType = ColumnMetaData.scalar(Types.INTEGER, "INTEGER", Rep.PRIMITIVE_INT);
+    // Two columns (types) in the ResultSet we will create
+    List<ColumnMetaData> types = Arrays.asList(ColumnMetaData.dummy(arrayOffsetType, false),
+        ColumnMetaData.dummy(elementType, true));
+    List<List<Object>> rows = createResultSetRowsForArrayData(elements);
+    // `(List<Object>) rows` is a compile error.
+    @SuppressWarnings({ "unchecked", "rawtypes" })
+    List<Object> untypedRows = (List<Object>) ((List) rows);
+    try (ListIteratorCursor cursor = new ListIteratorCursor(rows.iterator())) {
+      final String sql = "MOCKED";
+      QueryState state = new QueryState(sql);
+      Meta.Signature signature = new Meta.Signature(types, sql,
+          Collections.<AvaticaParameter>emptyList(), Collections.<String, Object>emptyMap(),
+          Meta.CursorFactory.LIST, Meta.StatementType.SELECT);
+      AvaticaResultSetMetaData resultSetMetaData = new AvaticaResultSetMetaData(null, sql,
+          signature);
+      Meta.Frame frame = new Meta.Frame(0, true, untypedRows);
+      AvaticaResultSet resultSet = new AvaticaResultSet(null, state, signature, resultSetMetaData,
+          timeZone, frame);
+      resultSet.execute2(cursor, types);
+      return resultSet;
+    }
+  }
+
+  @Override public Array createArray(AvaticaType elementType, Iterable<Object> elements) {
+    final ArrayType array = ColumnMetaData.array(elementType, elementType.name, Rep.ARRAY);
+    final List<ColumnMetaData> types = Collections.singletonList(ColumnMetaData.dummy(array, true));
+    // Avoid creating a new List if we already have a List
+    List<Object> elementList;
+    if (elements instanceof List) {
+      elementList = (List<Object>) elements;
+    } else {
+      elementList = new ArrayList<>();
+      for (Object element : elements) {
+        elementList.add(element);
+      }
+    }
+    try (ListIteratorCursor cursor = new ListIteratorCursor(createRowForArrayData(elementList))) {
+      List<Accessor> accessor = cursor.createAccessors(types, Unsafe.localCalendar(), this);
+      assert 1 == accessor.size();
+      return new ArrayImpl(elementList, (ArrayAccessor) accessor.get(0));
+    }
+  }
+
+  /**
+   * Creates the row-level view over the values that will make up an Array. The Iterator has a row
+   * per Array element, each row containing two columns. The second column is the array element and
+   * the first column is the offset into the array of that array element (one-based, not zero-based)
+   *
+   * The ordering of the rows is not guaranteed to be in the same order as the array elements.
+   *
+   * A list of {@code elements}:
+   * <pre>[1, 2, 3]</pre>
+   * might be converted into
+   * <pre>Iterator{ [1, 1], [2, 2], [3, 3] }</pre>
+   *
+   * @param elements The elements of an array.
+   */
+  private List<List<Object>> createResultSetRowsForArrayData(Iterable<Object> elements) {
+    List<List<Object>> rows = new ArrayList<>();
+    int i = 0;
+    for (Object element : elements) {
+      rows.add(Arrays.asList(i + 1, element));
+      i++;
+    }
+    return rows;
+  }
+
+  /**
+   * Creates an row-level view over the values that will make up an Array. The Iterator has one
+   * entry which has a list that also has one entry.
+   *
+   * A provided list of {@code elements}
+   * <pre>[1, 2, 3]</pre>
+   * would be converted into
+   * <pre>Iterator{ [ [1,2,3] ] }</pre>
+   *
+   * @param elements The elements of an array
+   */
+  private Iterator<List<Object>> createRowForArrayData(List<Object> elements) {
+    // Make a "row" with one "column" (which is really a list)
+    final List<Object> row = Collections.singletonList((Object) elements);
+    // Make an iterator over this one "row"
+    return Collections.singletonList(row).iterator();
+  }
+}
+
+// End ArrayFactoryImpl.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/util/ArrayImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/util/ArrayImpl.java b/core/src/main/java/org/apache/calcite/avatica/util/ArrayImpl.java
index b2d5ae9..e57fde8 100644
--- a/core/src/main/java/org/apache/calcite/avatica/util/ArrayImpl.java
+++ b/core/src/main/java/org/apache/calcite/avatica/util/ArrayImpl.java
@@ -27,18 +27,19 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+
 /** Implementation of JDBC {@link Array}. */
 public class ArrayImpl implements Array {
-  private final List list;
+  private final List<Object> list;
   private final AbstractCursor.ArrayAccessor accessor;
 
-  public ArrayImpl(List list, AbstractCursor.ArrayAccessor accessor) {
+  public ArrayImpl(List<Object> list, AbstractCursor.ArrayAccessor accessor) {
     this.list = list;
     this.accessor = accessor;
   }
 
   public String getBaseTypeName() throws SQLException {
-    return accessor.componentType.name;
+    return accessor.componentType.getName();
   }
 
   public int getBaseType() throws SQLException {
@@ -46,11 +47,11 @@ public class ArrayImpl implements Array {
   }
 
   public Object getArray() throws SQLException {
-    return getArray(list);
+    return getArray(list, accessor);
   }
 
   @Override public String toString() {
-    final Iterator iterator = list.iterator();
+    final Iterator<?> iterator = list.iterator();
     if (!iterator.hasNext()) {
       return "[]";
     }
@@ -93,9 +94,10 @@ public class ArrayImpl implements Array {
    * @throws NullPointerException if any element is null
    */
   @SuppressWarnings("unchecked")
-  protected Object getArray(List list) throws SQLException {
+  protected Object getArray(List<?> list, AbstractCursor.ArrayAccessor arrayAccessor)
+      throws SQLException {
     int i = 0;
-    switch (accessor.componentType.rep) {
+    switch (arrayAccessor.componentType.rep) {
     case PRIMITIVE_DOUBLE:
       final double[] doubles = new double[list.size()];
       for (double v : (List<Double>) list) {
@@ -148,56 +150,96 @@ public class ArrayImpl implements Array {
       // fall through
     }
     final Object[] objects = list.toArray();
-    switch (accessor.componentType.id) {
+    switch (arrayAccessor.componentType.id) {
     case Types.ARRAY:
       final AbstractCursor.ArrayAccessor componentAccessor =
-          (AbstractCursor.ArrayAccessor) accessor.componentAccessor;
+          (AbstractCursor.ArrayAccessor) arrayAccessor.componentAccessor;
       for (i = 0; i < objects.length; i++) {
-        objects[i] = new ArrayImpl((List) objects[i], componentAccessor);
+        // Convert the element into a Object[] or primitive array, recurse!
+        objects[i] = getArrayData(objects[i], componentAccessor);
       }
     }
     return objects;
   }
 
-  public Object getArray(Map<String, Class<?>> map) throws SQLException {
+  Object getArrayData(Object o, AbstractCursor.ArrayAccessor componentAccessor)
+      throws SQLException {
+    if (o instanceof List) {
+      return getArray((List<?>) o, componentAccessor);
+    } else if (o instanceof ArrayImpl) {
+      return (ArrayImpl) o;
+    }
+    throw new RuntimeException("Unhandled");
+  }
+
+  @Override 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));
+  @Override public Object getArray(long index, int count) throws SQLException {
+    if (index > Integer.MAX_VALUE) {
+      throw new IllegalArgumentException("Arrays cannot be longer than " + Integer.MAX_VALUE);
+    }
+    // Convert from one-index to zero-index
+    int startIndex = ((int) index) - 1;
+    if (startIndex < 0 || startIndex > list.size()) {
+      throw new IllegalArgumentException("Invalid index: " + index + ". Size = " + list.size());
+    }
+    int endIndex = startIndex + count;
+    if (endIndex > list.size()) {
+      throw new IllegalArgumentException("Invalid count provided. Size = " + list.size()
+          + ", count = " + count);
+    }
+    // End index is non-inclusive
+    return getArray(list.subList(startIndex, endIndex), accessor);
   }
 
-  public Object getArray(long index, int count, Map<String, Class<?>> map)
+  @Override public Object getArray(long index, int count, Map<String, Class<?>> map)
       throws SQLException {
     throw new UnsupportedOperationException(); // TODO
   }
 
-  public ResultSet getResultSet() throws SQLException {
+  @Override public ResultSet getResultSet() throws SQLException {
     return accessor.factory.create(accessor.componentType, list);
   }
 
-  public ResultSet getResultSet(Map<String, Class<?>> map)
+  @Override public ResultSet getResultSet(Map<String, Class<?>> map)
       throws SQLException {
     throw new UnsupportedOperationException(); // TODO
   }
 
-  public ResultSet getResultSet(long index, int count) throws SQLException {
+  @Override public ResultSet getResultSet(long index, int count) throws SQLException {
     throw new UnsupportedOperationException(); // TODO
   }
 
-  public ResultSet getResultSet(long index, int count,
+  @Override public ResultSet getResultSet(long index, int count,
       Map<String, Class<?>> map) throws SQLException {
     throw new UnsupportedOperationException(); // TODO
   }
 
-  public void free() throws SQLException {
+  @Override public void free() throws SQLException {
     // nothing to do
   }
 
-  /** Factory that can create a result set based on a list of values. */
+  /** Factory that can create a ResultSet or Array based on a stream of values. */
   public interface Factory {
-    ResultSet create(ColumnMetaData.AvaticaType elementType,
-        Iterable<Object> iterable);
+
+    /**
+     * Creates a {@link ResultSet} from the given list of values per {@link Array#getResultSet()}.
+     *
+     * @param elementType The type of the elements
+     * @param iterable The elements
+     */
+    ResultSet create(ColumnMetaData.AvaticaType elementType, Iterable<Object> iterable);
+
+    /**
+     * Creates an {@link Array} from the given list of values, converting any primitive values
+     * into the corresponding objects.
+     *
+     * @param elementType The type of the elements
+     * @param elements The elements
+     */
+    Array createArray(ColumnMetaData.AvaticaType elementType, Iterable<Object> elements);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/util/PositionedCursor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/util/PositionedCursor.java b/core/src/main/java/org/apache/calcite/avatica/util/PositionedCursor.java
index f60f47d..070319d 100644
--- a/core/src/main/java/org/apache/calcite/avatica/util/PositionedCursor.java
+++ b/core/src/main/java/org/apache/calcite/avatica/util/PositionedCursor.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.avatica.util;
 
 import java.lang.reflect.Field;
+import java.sql.SQLException;
 import java.util.List;
 import java.util.Map;
 
@@ -48,7 +49,19 @@ public abstract class PositionedCursor<T> extends AbstractCursor {
     }
 
     public Object getObject() {
-      Object o = ((Object[]) current())[field];
+      Object collection = current();
+      Object o;
+      if (collection instanceof List) {
+        o = ((List) collection).get(field);
+      } else if (collection instanceof StructImpl) {
+        try {
+          o = ((StructImpl) collection).getAttributes()[field];
+        } catch (SQLException e) {
+          throw new RuntimeException(e);
+        }
+      } else {
+        o = ((Object[]) collection)[field];
+      }
       wasNull[0] = o == null;
       return o;
     }

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/java/org/apache/calcite/avatica/util/Unsafe.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/util/Unsafe.java b/core/src/main/java/org/apache/calcite/avatica/util/Unsafe.java
index 1d0238c..906651d 100644
--- a/core/src/main/java/org/apache/calcite/avatica/util/Unsafe.java
+++ b/core/src/main/java/org/apache/calcite/avatica/util/Unsafe.java
@@ -50,6 +50,17 @@ public class Unsafe {
   public static Calendar localCalendar() {
     return Calendar.getInstance(Locale.ROOT);
   }
+
+  /**
+   * Returns a {@link java.lang.String}, created from the given format and args,
+   * with the root locale. Analog to {@link String#format(String, Object...)}.
+   *
+   * @param format The format string
+   * @param args Arguments to be substituted into the format string.
+   */
+  public static String formatLocalString(String format, Object... args) {
+    return String.format(Locale.ROOT, format, args);
+  }
 }
 
 // End Unsafe.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/main/protobuf/common.proto
----------------------------------------------------------------------
diff --git a/core/src/main/protobuf/common.proto b/core/src/main/protobuf/common.proto
index affe5d5..63dbcc9 100644
--- a/core/src/main/protobuf/common.proto
+++ b/core/src/main/protobuf/common.proto
@@ -199,6 +199,9 @@ message TypedValue {
   bytes bytes_value = 5; // binary/varbinary
   double double_value = 6; // big numbers
   bool null = 7; // a null object
+
+  repeated TypedValue array_value = 8; // The Array
+  Rep component_type = 9; // If an Array, the representation for the array values
 }
 
 // The severity of some unexpected outcome to an operation.

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/test/java/org/apache/calcite/avatica/AvaticaResultSetConversionsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/avatica/AvaticaResultSetConversionsTest.java b/core/src/test/java/org/apache/calcite/avatica/AvaticaResultSetConversionsTest.java
index bf3047f..8605aaf 100644
--- a/core/src/test/java/org/apache/calcite/avatica/AvaticaResultSetConversionsTest.java
+++ b/core/src/test/java/org/apache/calcite/avatica/AvaticaResultSetConversionsTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.calcite.avatica;
 
-import org.apache.calcite.avatica.ColumnMetaData.AvaticaType;
 import org.apache.calcite.avatica.remote.TypedValue;
 import org.apache.calcite.avatica.util.DateTimeUtils;
 
@@ -85,22 +84,11 @@ public class AvaticaResultSetConversionsTest {
       throw new UnsupportedOperationException();
     }
 
-    @SuppressWarnings("deprecation")
     @Override public ExecuteResult prepareAndExecute(StatementHandle h, String sql,
         long maxRowCount, PrepareCallback callback) throws NoSuchStatementException {
       throw new UnsupportedOperationException();
     }
 
-    private static ColumnMetaData columnMetaData(String name, int ordinal, AvaticaType type,
-        int columnNullable) {
-      return new ColumnMetaData(
-          ordinal, false, true, false, false,
-          columnNullable,
-          true, -1, name, name, null,
-          0, 0, null, null, type, true, false, false,
-          type.columnClassName());
-    }
-
     @Override public ExecuteResult prepareAndExecute(StatementHandle h, String sql,
         long maxRowCount, int maxRowsInFirstFrame, PrepareCallback callback)
         throws NoSuchStatementException {
@@ -191,7 +179,6 @@ public class AvaticaResultSetConversionsTest {
       throw new UnsupportedOperationException();
     }
 
-    @SuppressWarnings("deprecation")
     @Override public ExecuteResult execute(StatementHandle h, List<TypedValue> parameterValues,
         long maxRowCount) throws NoSuchStatementException {
       throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/test/java/org/apache/calcite/avatica/FrameTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/avatica/FrameTest.java b/core/src/test/java/org/apache/calcite/avatica/FrameTest.java
index e17bf92..4f34a3c 100644
--- a/core/src/test/java/org/apache/calcite/avatica/FrameTest.java
+++ b/core/src/test/java/org/apache/calcite/avatica/FrameTest.java
@@ -206,6 +206,34 @@ public class FrameTest {
     List<Common.TypedValue> arrayValues = protoColumns.get(1).getArrayValueList();
     assertEquals(arrayValues, deprecatedValues);
   }
+
+  @Test public void testNestedArraySerialization() {
+    List<Object> rows = new ArrayList<>();
+    // [ "pk", [[1,2], [3,4]] ]
+    rows.add(Arrays.asList("pk", Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4))));
+    Frame frame = new Frame(0, true, rows);
+    // Parse back the list in serialized form
+    Common.Frame protoFrame = frame.toProto();
+    Common.Row protoRow = protoFrame.getRows(0);
+    Common.ColumnValue protoColumn = protoRow.getValue(1);
+    assertTrue(protoColumn.getHasArrayValue());
+    int value = 1;
+    for (Common.TypedValue arrayElement : protoColumn.getArrayValueList()) {
+      assertEquals(Common.Rep.ARRAY, arrayElement.getType());
+      for (Common.TypedValue nestedArrayElement : arrayElement.getArrayValueList()) {
+        assertEquals(Common.Rep.INTEGER, nestedArrayElement.getType());
+        assertEquals(value++, nestedArrayElement.getNumberValue());
+      }
+    }
+
+    Frame newFrame = Frame.fromProto(protoFrame);
+    @SuppressWarnings("unchecked")
+    List<Object> newRow = (List<Object>) newFrame.rows.iterator().next();
+    @SuppressWarnings("unchecked")
+    List<Object> expectedRow = (List<Object>) rows.get(0);
+    assertEquals(expectedRow.get(0), newRow.get(0));
+    assertEquals(expectedRow.get(1), newRow.get(1));
+  }
 }
 
 // End FrameTest.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/test/java/org/apache/calcite/avatica/RepTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/avatica/RepTest.java b/core/src/test/java/org/apache/calcite/avatica/RepTest.java
new file mode 100644
index 0000000..ce0ba9b
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/avatica/RepTest.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.avatica;
+
+import org.apache.calcite.avatica.ColumnMetaData.Rep;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test class for {@link Rep}.
+ */
+public class RepTest {
+
+  @Test public void testNonPrimitiveRepForType() {
+    assertEquals(Rep.BOOLEAN, Rep.nonPrimitiveRepOf(SqlType.BIT));
+    assertEquals(Rep.BOOLEAN, Rep.nonPrimitiveRepOf(SqlType.BOOLEAN));
+    assertEquals(Rep.BYTE, Rep.nonPrimitiveRepOf(SqlType.TINYINT));
+    assertEquals(Rep.SHORT, Rep.nonPrimitiveRepOf(SqlType.SMALLINT));
+    assertEquals(Rep.INTEGER, Rep.nonPrimitiveRepOf(SqlType.INTEGER));
+    assertEquals(Rep.LONG, Rep.nonPrimitiveRepOf(SqlType.BIGINT));
+    assertEquals(Rep.DOUBLE, Rep.nonPrimitiveRepOf(SqlType.FLOAT));
+    assertEquals(Rep.DOUBLE, Rep.nonPrimitiveRepOf(SqlType.DOUBLE));
+    assertEquals(Rep.STRING, Rep.nonPrimitiveRepOf(SqlType.CHAR));
+  }
+
+  @Test public void testSerialRep() {
+    assertEquals(Rep.BOOLEAN, Rep.serialRepOf(SqlType.BIT));
+    assertEquals(Rep.BOOLEAN, Rep.serialRepOf(SqlType.BOOLEAN));
+    assertEquals(Rep.BYTE, Rep.serialRepOf(SqlType.TINYINT));
+    assertEquals(Rep.SHORT, Rep.serialRepOf(SqlType.SMALLINT));
+    assertEquals(Rep.INTEGER, Rep.serialRepOf(SqlType.INTEGER));
+    assertEquals(Rep.LONG, Rep.serialRepOf(SqlType.BIGINT));
+    assertEquals(Rep.DOUBLE, Rep.serialRepOf(SqlType.FLOAT));
+    assertEquals(Rep.DOUBLE, Rep.serialRepOf(SqlType.DOUBLE));
+    assertEquals(Rep.INTEGER, Rep.serialRepOf(SqlType.DATE));
+    assertEquals(Rep.INTEGER, Rep.serialRepOf(SqlType.TIME));
+    assertEquals(Rep.LONG, Rep.serialRepOf(SqlType.TIMESTAMP));
+  }
+}
+
+// End RepTest.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/test/java/org/apache/calcite/avatica/remote/TypedValueTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/avatica/remote/TypedValueTest.java b/core/src/test/java/org/apache/calcite/avatica/remote/TypedValueTest.java
index 7606a87..50d492e 100644
--- a/core/src/test/java/org/apache/calcite/avatica/remote/TypedValueTest.java
+++ b/core/src/test/java/org/apache/calcite/avatica/remote/TypedValueTest.java
@@ -16,16 +16,25 @@
  */
 package org.apache.calcite.avatica.remote;
 
+import org.apache.calcite.avatica.ColumnMetaData;
 import org.apache.calcite.avatica.ColumnMetaData.Rep;
+import org.apache.calcite.avatica.ColumnMetaData.ScalarType;
 import org.apache.calcite.avatica.proto.Common;
+import org.apache.calcite.avatica.util.ArrayFactoryImpl;
+import org.apache.calcite.avatica.util.ArrayImpl;
 import org.apache.calcite.avatica.util.Base64;
 import org.apache.calcite.avatica.util.ByteString;
 import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.Unsafe;
 
 import org.junit.Test;
 
 import java.math.BigDecimal;
+import java.sql.Array;
+import java.sql.Types;
+import java.util.Arrays;
 import java.util.Calendar;
+import java.util.List;
 
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
@@ -33,6 +42,7 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
@@ -203,6 +213,25 @@ public class TypedValueTest {
     assertEquals(Rep.BYTE_STRING, tv.type);
     assertEquals(base64Str, tv.value);
   }
+
+  @Test public void testArrays() {
+    List<Object> serialObj = Arrays.<Object>asList(1, 2, 3, 4);
+    ArrayImpl.Factory factory = new ArrayFactoryImpl(Unsafe.localCalendar().getTimeZone());
+    ScalarType scalarType = ColumnMetaData.scalar(Types.INTEGER, "INTEGER", Rep.INTEGER);
+    Array a1 = factory.createArray(scalarType, serialObj);
+    TypedValue tv1 = TypedValue.ofJdbc(Rep.ARRAY, a1, Unsafe.localCalendar());
+    Object jdbcObj = tv1.toJdbc(Unsafe.localCalendar());
+    assertTrue("The JDBC object is an " + jdbcObj.getClass(), jdbcObj instanceof Array);
+    Object localObj = tv1.toLocal();
+    assertTrue("The local object is an " + localObj.getClass(), localObj instanceof List);
+    Common.TypedValue protoTv1 = tv1.toProto();
+    assertEquals(serialObj.size(), protoTv1.getArrayValueCount());
+    TypedValue tv1Copy = TypedValue.fromProto(protoTv1);
+    Object jdbcObjCopy = tv1Copy.toJdbc(Unsafe.localCalendar());
+    assertTrue("The JDBC object is an " + jdbcObjCopy.getClass(), jdbcObjCopy instanceof Array);
+    Object localObjCopy = tv1Copy.toLocal();
+    assertTrue("The local object is an " + localObjCopy.getClass(), localObjCopy instanceof List);
+  }
 }
 
 // End TypedValueTest.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/test/java/org/apache/calcite/avatica/util/ArrayImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/avatica/util/ArrayImplTest.java b/core/src/test/java/org/apache/calcite/avatica/util/ArrayImplTest.java
new file mode 100644
index 0000000..2ebd13b
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/avatica/util/ArrayImplTest.java
@@ -0,0 +1,193 @@
+/*
+ * 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 org.apache.calcite.avatica.ColumnMetaData.ArrayType;
+import org.apache.calcite.avatica.ColumnMetaData.Rep;
+import org.apache.calcite.avatica.ColumnMetaData.ScalarType;
+import org.apache.calcite.avatica.ColumnMetaData.StructType;
+import org.apache.calcite.avatica.MetaImpl;
+import org.apache.calcite.avatica.util.Cursor.Accessor;
+
+import org.junit.Test;
+
+import java.sql.Array;
+import java.sql.ResultSet;
+import java.sql.Struct;
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test class for ArrayImpl.
+ */
+public class ArrayImplTest {
+
+  @Test public void resultSetFromArray() throws Exception {
+    // Define the struct type we're creating
+    ScalarType intType = ColumnMetaData.scalar(Types.INTEGER, "INTEGER", Rep.INTEGER);
+    ArrayType arrayType = ColumnMetaData.array(intType, "INTEGER", Rep.INTEGER);
+    ColumnMetaData arrayMetaData = MetaImpl.columnMetaData("MY_ARRAY", 1, arrayType, false);
+    ArrayImpl.Factory factory = new ArrayFactoryImpl(Unsafe.localCalendar().getTimeZone());
+    // Create some arrays from the structs
+    Array array1 = factory.createArray(intType, Arrays.<Object>asList(1, 2));
+    Array array2 = factory.createArray(intType, Arrays.<Object>asList(3));
+    Array array3 = factory.createArray(intType, Arrays.<Object>asList(4, 5, 6));
+    List<List<Object>> rows = Arrays.asList(Collections.<Object>singletonList(array1),
+        Collections.<Object>singletonList(array2), Collections.<Object>singletonList(array3));
+    // Create two rows, each with one (array) column
+    try (Cursor cursor = new ListIteratorCursor(rows.iterator())) {
+      List<Accessor> accessors = cursor.createAccessors(Collections.singletonList(arrayMetaData),
+          Unsafe.localCalendar(), factory);
+      assertEquals(1, accessors.size());
+      Accessor accessor = accessors.get(0);
+
+      assertTrue(cursor.next());
+      Array actualArray = accessor.getArray();
+      // An Array's result set has one row per array element.
+      // Each row has two columns. Column 1 is the array offset (1-based), Column 2 is the value.
+      ResultSet actualArrayResultSet = actualArray.getResultSet();
+      assertEquals(2, actualArrayResultSet.getMetaData().getColumnCount());
+      assertTrue(actualArrayResultSet.next());
+      // Order is Avatica implementation specific
+      assertEquals(1, actualArrayResultSet.getInt(1));
+      assertEquals(1, actualArrayResultSet.getInt(2));
+      assertTrue(actualArrayResultSet.next());
+      assertEquals(2, actualArrayResultSet.getInt(1));
+      assertEquals(2, actualArrayResultSet.getInt(2));
+      assertFalse(actualArrayResultSet.next());
+
+      assertTrue(cursor.next());
+      actualArray = accessor.getArray();
+      actualArrayResultSet = actualArray.getResultSet();
+      assertEquals(2, actualArrayResultSet.getMetaData().getColumnCount());
+      assertTrue(actualArrayResultSet.next());
+      assertEquals(1, actualArrayResultSet.getInt(1));
+      assertEquals(3, actualArrayResultSet.getInt(2));
+      assertFalse(actualArrayResultSet.next());
+
+      assertTrue(cursor.next());
+      actualArray = accessor.getArray();
+      actualArrayResultSet = actualArray.getResultSet();
+      assertEquals(2, actualArrayResultSet.getMetaData().getColumnCount());
+      assertTrue(actualArrayResultSet.next());
+      assertEquals(1, actualArrayResultSet.getInt(1));
+      assertEquals(4, actualArrayResultSet.getInt(2));
+      assertTrue(actualArrayResultSet.next());
+      assertEquals(2, actualArrayResultSet.getInt(1));
+      assertEquals(5, actualArrayResultSet.getInt(2));
+      assertTrue(actualArrayResultSet.next());
+      assertEquals(3, actualArrayResultSet.getInt(1));
+      assertEquals(6, actualArrayResultSet.getInt(2));
+      assertFalse(actualArrayResultSet.next());
+
+      assertFalse(cursor.next());
+    }
+  }
+
+  @Test public void arraysOfStructs() throws Exception {
+    // Define the struct type we're creating
+    ColumnMetaData intMetaData = MetaImpl.columnMetaData("MY_INT", 1, int.class, false);
+    ColumnMetaData stringMetaData = MetaImpl.columnMetaData("MY_STRING", 2, String.class, true);
+    StructType structType = ColumnMetaData.struct(Arrays.asList(intMetaData, stringMetaData));
+    // Create some structs
+    Struct struct1 = new StructImpl(Arrays.<Object>asList(1, "one"));
+    Struct struct2 = new StructImpl(Arrays.<Object>asList(2, "two"));
+    Struct struct3 = new StructImpl(Arrays.<Object>asList(3));
+    Struct struct4 = new StructImpl(Arrays.<Object>asList(4, "four"));
+    ArrayType arrayType = ColumnMetaData.array(structType, "OBJECT", Rep.STRUCT);
+    ColumnMetaData arrayMetaData = MetaImpl.columnMetaData("MY_ARRAY", 1, arrayType, false);
+    ArrayImpl.Factory factory = new ArrayFactoryImpl(Unsafe.localCalendar().getTimeZone());
+    // Create some arrays from the structs
+    Array array1 = factory.createArray(structType, Arrays.<Object>asList(struct1, struct2));
+    Array array2 = factory.createArray(structType, Arrays.<Object>asList(struct3, struct4));
+    List<List<Object>> rows = Arrays.asList(Collections.<Object>singletonList(array1),
+        Collections.<Object>singletonList(array2));
+    // Create two rows, each with one (array) column
+    try (Cursor cursor = new ListIteratorCursor(rows.iterator())) {
+      List<Accessor> accessors = cursor.createAccessors(Collections.singletonList(arrayMetaData),
+          Unsafe.localCalendar(), factory);
+      assertEquals(1, accessors.size());
+      Accessor accessor = accessors.get(0);
+
+      assertTrue(cursor.next());
+      Array actualArray = accessor.getArray();
+      // Avoiding explicit use of the getResultSet() method for now..
+      Object[] arrayData = (Object[]) actualArray.getArray();
+      assertEquals(2, arrayData.length);
+      Struct actualStruct = (Struct) arrayData[0];
+      Object[] o = actualStruct.getAttributes();
+      assertEquals(2, o.length);
+      assertEquals(1, o[0]);
+      assertEquals("one", o[1]);
+
+      actualStruct = (Struct) arrayData[1];
+      o = actualStruct.getAttributes();
+      assertEquals(2, o.length);
+      assertEquals(2, o[0]);
+      assertEquals("two", o[1]);
+
+      assertTrue(cursor.next());
+      actualArray = accessor.getArray();
+      arrayData = (Object[]) actualArray.getArray();
+      assertEquals(2, arrayData.length);
+      actualStruct = (Struct) arrayData[0];
+      o = actualStruct.getAttributes();
+      assertEquals(1, o.length);
+      assertEquals(3, o[0]);
+
+      actualStruct = (Struct) arrayData[1];
+      o = actualStruct.getAttributes();
+      assertEquals(2, o.length);
+      assertEquals(4, o[0]);
+      assertEquals("four", o[1]);
+    }
+  }
+
+  @Test public void testArrayWithOffsets() throws Exception {
+    // Define the struct type we're creating
+    ScalarType intType = ColumnMetaData.scalar(Types.INTEGER, "INTEGER", Rep.INTEGER);
+    ArrayImpl.Factory factory = new ArrayFactoryImpl(Unsafe.localCalendar().getTimeZone());
+    // Create some arrays from the structs
+    Array array1 = factory.createArray(intType, Arrays.<Object>asList(1, 2));
+    Array array3 = factory.createArray(intType, Arrays.<Object>asList(4, 5, 6));
+
+    Object[] data = (Object[]) array1.getArray(2, 1);
+    assertEquals(1, data.length);
+    assertEquals(2, data[0]);
+    data = (Object[]) array3.getArray(1, 1);
+    assertEquals(1, data.length);
+    assertEquals(4, data[0]);
+    data = (Object[]) array3.getArray(2, 2);
+    assertEquals(2, data.length);
+    assertEquals(5, data[0]);
+    assertEquals(6, data[1]);
+    data = (Object[]) array3.getArray(1, 3);
+    assertEquals(3, data.length);
+    assertEquals(4, data[0]);
+    assertEquals(5, data[1]);
+    assertEquals(6, data[2]);
+  }
+}
+
+// End ArrayImplTest.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/core/src/test/java/org/apache/calcite/avatica/util/StructImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/avatica/util/StructImplTest.java b/core/src/test/java/org/apache/calcite/avatica/util/StructImplTest.java
new file mode 100644
index 0000000..625c538
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/avatica/util/StructImplTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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 org.apache.calcite.avatica.ColumnMetaData.StructType;
+import org.apache.calcite.avatica.MetaImpl;
+import org.apache.calcite.avatica.util.Cursor.Accessor;
+
+import org.junit.Test;
+
+import java.sql.Struct;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test class for StructImpl.
+ */
+public class StructImplTest {
+
+  @Test public void structAccessor() throws Exception {
+    // Define the struct type we're creating
+    ColumnMetaData intMetaData = MetaImpl.columnMetaData("MY_INT", 1, int.class, false);
+    ColumnMetaData stringMetaData = MetaImpl.columnMetaData("MY_STRING", 2, String.class, true);
+    StructType structType = ColumnMetaData.struct(Arrays.asList(intMetaData, stringMetaData));
+    // Create some structs
+    Struct struct1 = new StructImpl(Arrays.<Object>asList(1, "one"));
+    Struct struct2 = new StructImpl(Arrays.<Object>asList(2, "two"));
+    Struct struct3 = new StructImpl(Arrays.<Object>asList(3));
+    Struct struct4 = new StructImpl(Arrays.<Object>asList(4, "four", "ignored"));
+    ColumnMetaData structMetaData = MetaImpl.columnMetaData("MY_STRUCT", 1, structType, false);
+    List<List<Object>> rows = Arrays.asList(Collections.<Object>singletonList(struct1),
+        Collections.<Object>singletonList(struct2), Collections.<Object>singletonList(struct3),
+        Collections.<Object>singletonList(struct4));
+    // Create four rows, each with one (struct) column
+    try (Cursor cursor = new ListIteratorCursor(rows.iterator())) {
+      List<Accessor> accessors = cursor.createAccessors(Collections.singletonList(structMetaData),
+          Unsafe.localCalendar(), null);
+      assertEquals(1, accessors.size());
+      Accessor accessor = accessors.get(0);
+
+      assertTrue(cursor.next());
+      Struct s = accessor.getObject(Struct.class);
+      Object[] structData = s.getAttributes();
+      assertEquals(2, structData.length);
+      assertEquals(1, structData[0]);
+      assertEquals("one", structData[1]);
+
+      assertTrue(cursor.next());
+      s = accessor.getObject(Struct.class);
+      structData = s.getAttributes();
+      assertEquals(2, structData.length);
+      assertEquals(2, structData[0]);
+      assertEquals("two", structData[1]);
+
+      assertTrue(cursor.next());
+      s = accessor.getObject(Struct.class);
+      structData = s.getAttributes();
+      assertEquals(1, structData.length);
+      assertEquals(3, structData[0]);
+
+      assertTrue(cursor.next());
+      s = accessor.getObject(Struct.class);
+      structData = s.getAttributes();
+      assertEquals(3, structData.length);
+      assertEquals(4, structData[0]);
+      assertEquals("four", structData[1]);
+      // We didn't provide metadata, but we still expect to see it.
+      assertEquals("ignored", structData[2]);
+    }
+  }
+}
+
+// End StructImplTest.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java b/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
index 4756e8d..a02aa69 100644
--- a/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
+++ b/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
@@ -37,6 +37,7 @@ import org.apache.calcite.avatica.remote.ProtobufMeta;
 import org.apache.calcite.avatica.remote.TypedValue;
 import org.apache.calcite.avatica.util.Unsafe;
 
+import com.google.common.base.Optional;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.RemovalListener;
@@ -805,7 +806,7 @@ public class JdbcMeta implements ProtobufMeta {
         return Frame.EMPTY;
       } else {
         return JdbcResultSet.frame(statementInfo, statementInfo.getResultSet(), offset,
-            fetchMaxRowCount, calendar);
+            fetchMaxRowCount, calendar, Optional.<Meta.Signature>absent());
       }
     } catch (SQLException e) {
       throw propagate(e);
@@ -819,7 +820,6 @@ public class JdbcMeta implements ProtobufMeta {
     return typeList.toArray(new String[typeList.size()]);
   }
 
-  @SuppressWarnings("deprecation")
   @Override public ExecuteResult execute(StatementHandle h, List<TypedValue> parameterValues,
       long maxRowCount) throws NoSuchStatementException {
     return execute(h, parameterValues, AvaticaUtils.toSaturatedInt(maxRowCount));
@@ -848,7 +848,6 @@ public class JdbcMeta implements ProtobufMeta {
       }
 
       if (preparedStatement.execute()) {
-        final Meta.Frame frame;
         final Signature signature2;
         if (preparedStatement.isWrapperFor(AvaticaPreparedStatement.class)) {
           signature2 = h.signature;
@@ -863,7 +862,6 @@ public class JdbcMeta implements ProtobufMeta {
         statementInfo.setResultSet(preparedStatement.getResultSet());
 
         if (statementInfo.getResultSet() == null) {
-          frame = Frame.EMPTY;
           resultSets = Collections.<MetaResultSet>singletonList(
               JdbcResultSet.empty(h.connectionId, h.id, signature2));
         } else {

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java b/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
index 17b33f8..b879086 100644
--- a/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
+++ b/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcResultSet.java
@@ -17,21 +17,31 @@
 package org.apache.calcite.avatica.jdbc;
 
 import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.AvaticaUtils;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.ColumnMetaData.ArrayType;
+import org.apache.calcite.avatica.ColumnMetaData.AvaticaType;
 import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.SqlType;
 import org.apache.calcite.avatica.util.DateTimeUtils;
 
+import com.google.common.base.Optional;
+
 import java.sql.Array;
 import java.sql.Date;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
 import java.sql.Struct;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.sql.Types;
 import java.util.ArrayList;
 import java.util.Calendar;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import java.util.TreeMap;
 
 /** Implementation of {@link org.apache.calcite.avatica.Meta.MetaResultSet}
@@ -88,7 +98,8 @@ class JdbcResultSet extends Meta.MetaResultSet {
       } else {
         fetchRowCount = maxRowCount;
       }
-      final Meta.Frame firstFrame = frame(null, resultSet, 0, fetchRowCount, calendar);
+      final Meta.Frame firstFrame = frame(null, resultSet, 0, fetchRowCount, calendar,
+          Optional.of(signature));
       if (firstFrame.done) {
         resultSet.close();
       }
@@ -115,12 +126,16 @@ class JdbcResultSet extends Meta.MetaResultSet {
   /** Creates a frame containing a given number or unlimited number of rows
    * from a result set. */
   static Meta.Frame frame(StatementInfo info, ResultSet resultSet, long offset,
-      int fetchMaxRowCount, Calendar calendar) throws SQLException {
+      int fetchMaxRowCount, Calendar calendar, Optional<Meta.Signature> sig) throws SQLException {
     final ResultSetMetaData metaData = resultSet.getMetaData();
     final int columnCount = metaData.getColumnCount();
     final int[] types = new int[columnCount];
+    Set<Integer> arrayOffsets = new HashSet<>();
     for (int i = 0; i < types.length; i++) {
       types[i] = metaData.getColumnType(i + 1);
+      if (Types.ARRAY == types[i]) {
+        arrayOffsets.add(i);
+      }
     }
     final List<Object> rows = new ArrayList<>();
     // Meta prepare/prepareAndExecute 0 return 0 row and done
@@ -140,6 +155,28 @@ class JdbcResultSet extends Meta.MetaResultSet {
       Object[] columns = new Object[columnCount];
       for (int j = 0; j < columnCount; j++) {
         columns[j] = getValue(resultSet, types[j], j, calendar);
+        if (arrayOffsets.contains(j)) {
+          // If we have an Array type, our Signature is lacking precision. We can't extract the
+          // component type of an Array from metadata, we have to update it as we're serializing
+          // the ResultSet.
+          final Array array = resultSet.getArray(j + 1);
+          // Only attempt to determine the component type for the array when non-null
+          if (null != array && sig.isPresent()) {
+            ColumnMetaData columnMetaData = sig.get().columns.get(j);
+            ArrayType arrayType = (ArrayType) columnMetaData.type;
+            SqlType componentSqlType = SqlType.valueOf(array.getBaseType());
+
+            // Avatica Server will always return non-primitives to ensure nullable is guaranteed.
+            ColumnMetaData.Rep rep = ColumnMetaData.Rep.serialRepOf(componentSqlType);
+            AvaticaType componentType = ColumnMetaData.scalar(array.getBaseType(),
+                array.getBaseTypeName(), rep);
+            // Update the ArrayType from the Signature
+            arrayType.updateComponentType(componentType);
+
+            // We only need to update the array's type once.
+            arrayOffsets.remove(j);
+          }
+        }
       }
       rows.add(columns);
     }
@@ -186,18 +223,14 @@ class JdbcResultSet extends Meta.MetaResultSet {
       if (null == array) {
         return null;
       }
-      ResultSet arrayValues = array.getResultSet();
-      TreeMap<Integer, Object> map = new TreeMap<>();
-      while (arrayValues.next()) {
-        // column 1 is the index in the array, column 2 is the value.
-        // Recurse on `getValue` to unwrap nested types correctly.
-        // `j` is zero-indexed and incremented for us, thus we have `1` being used twice.
-        map.put(arrayValues.getInt(1), getValue(arrayValues, array.getBaseType(), 1, calendar));
+      try {
+        // Recursively extracts an Array using its ResultSet-representation
+        return extractUsingResultSet(array, calendar);
+      } catch (UnsupportedOperationException | SQLFeatureNotSupportedException e) {
+        // Not every database might implement Array.getResultSet(). This call
+        // assumes a non-nested array (depends on the db if that's a valid assumption)
+        return extractUsingArray(array, calendar);
       }
-      // If the result set is not in the same order as the actual Array, TreeMap fixes that.
-      // Need to make a concrete list to ensure Jackson serialization.
-      //return new ListLike<Object>(new ArrayList<>(map.values()), ListLikeType.ARRAY);
-      return new ArrayList<>(map.values());
     case Types.STRUCT:
       Struct struct = resultSet.getObject(j + 1, Struct.class);
       Object[] attrs = struct.getAttributes();
@@ -210,6 +243,39 @@ class JdbcResultSet extends Meta.MetaResultSet {
       return resultSet.getObject(j + 1);
     }
   }
+
+  /**
+   * Converts an Array into a List using {@link Array#getResultSet()}. This implementation is
+   * recursive and can parse multi-dimensional arrays.
+   */
+  static List<?> extractUsingResultSet(Array array, Calendar calendar) throws SQLException {
+    ResultSet arrayValues = array.getResultSet();
+    TreeMap<Integer, Object> map = new TreeMap<>();
+    while (arrayValues.next()) {
+      // column 1 is the index in the array, column 2 is the value.
+      // Recurse on `getValue` to unwrap nested types correctly.
+      // `j` is zero-indexed and incremented for us, thus we have `1` being used twice.
+      map.put(arrayValues.getInt(1), getValue(arrayValues, array.getBaseType(), 1, calendar));
+    }
+    // If the result set is not in the same order as the actual Array, TreeMap fixes that.
+    // Need to make a concrete list to ensure Jackson serialization.
+    return new ArrayList<>(map.values());
+  }
+
+  /**
+   * Converts an Array into a List using {@link Array#getArray()}. This implementation assumes
+   * a non-nested array. Use {link {@link #extractUsingResultSet(Array, Calendar)} if nested
+   * arrays may be possible.
+   */
+  static List<?> extractUsingArray(Array array, Calendar calendar) throws SQLException {
+    // No option but to guess as to what the type actually is...
+    Object o = array.getArray();
+    if (o instanceof List) {
+      return (List<?>) o;
+    }
+    // Assume that it's a Java array.
+    return AvaticaUtils.primitiveList(o);
+  }
 }
 
 // End JdbcResultSet.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java b/server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
index 0823a12..a69fa21 100644
--- a/server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
@@ -138,7 +138,7 @@ public class RemoteDriverTest {
   }
 
   // Run each test with the LocalJsonService and LocalProtobufService
-  @Parameters
+  @Parameters(name = "{0}")
   public static List<Object[]> parameters() {
     List<Object[]> connections = new ArrayList<>();
 
@@ -147,6 +147,7 @@ public class RemoteDriverTest {
 
     connections.add(
       new Object[] {
+        "JSON",
         new Callable<Connection>() {
           public Connection call() {
             try {
@@ -167,6 +168,7 @@ public class RemoteDriverTest {
     // TODO write the ConnectionInternals implementation
     connections.add(
       new Object[] {
+        "PROTOBUF",
         new Callable<Connection>() {
           public Connection call() {
             try {
@@ -191,7 +193,7 @@ public class RemoteDriverTest {
   private final ConnectionInternals localConnectionInternals;
   private final Callable<RequestInspection> requestInspectionCallable;
 
-  public RemoteDriverTest(Callable<Connection> localConnectionCallable,
+  public RemoteDriverTest(String name, Callable<Connection> localConnectionCallable,
       ConnectionInternals internals, Callable<RequestInspection> requestInspectionCallable) {
     this.localConnectionCallable = localConnectionCallable;
     this.localConnectionInternals = internals;
@@ -884,6 +886,7 @@ public class RemoteDriverTest {
       final ResultSet resultSet = ps.executeQuery();
       fail("expected error, got " + resultSet);
     } catch (SQLException e) {
+      LOG.info("Caught expected error", e);
       assertThat(e.getMessage(),
           containsString("exception while executing query: unbound parameter"));
     }

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/dd65a2b1/server/src/test/java/org/apache/calcite/avatica/remote/ArrayTypeTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/avatica/remote/ArrayTypeTest.java b/server/src/test/java/org/apache/calcite/avatica/remote/ArrayTypeTest.java
new file mode 100644
index 0000000..e1c3355
--- /dev/null
+++ b/server/src/test/java/org/apache/calcite/avatica/remote/ArrayTypeTest.java
@@ -0,0 +1,626 @@
+/*
+ * 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.AvaticaUtils;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.ColumnMetaData.ArrayType;
+import org.apache.calcite.avatica.ColumnMetaData.AvaticaType;
+import org.apache.calcite.avatica.ColumnMetaData.Rep;
+import org.apache.calcite.avatica.ColumnMetaData.ScalarType;
+import org.apache.calcite.avatica.SqlType;
+import org.apache.calcite.avatica.remote.Driver.Serialization;
+import org.apache.calcite.avatica.server.HttpServer;
+import org.apache.calcite.avatica.util.AbstractCursor.ArrayAccessor;
+import org.apache.calcite.avatica.util.ArrayImpl;
+import org.apache.calcite.avatica.util.Cursor.Accessor;
+import org.apache.calcite.avatica.util.ListIteratorCursor;
+import org.apache.calcite.avatica.util.Unsafe;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import java.sql.Array;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * Test class for verifying functionality with arrays.
+ */
+@RunWith(Parameterized.class)
+public class ArrayTypeTest {
+  private static final AvaticaServersForTest SERVERS = new AvaticaServersForTest();
+
+  private final HttpServer server;
+  private final String url;
+  private final int port;
+  @SuppressWarnings("unused")
+  private final Driver.Serialization serialization;
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> parameters() throws Exception {
+    SERVERS.startServers();
+    return SERVERS.getJUnitParameters();
+  }
+
+  public ArrayTypeTest(Serialization serialization, HttpServer server) {
+    this.server = server;
+    this.port = this.server.getPort();
+    this.serialization = serialization;
+    this.url = SERVERS.getJdbcUrl(port, serialization);
+  }
+
+  @AfterClass public static void afterClass() throws Exception {
+    if (null != SERVERS) {
+      SERVERS.stopServers();
+    }
+  }
+
+  @Test public void simpleArrayTest() throws Exception {
+    try (Connection conn = DriverManager.getConnection(url)) {
+      ScalarType varcharComponent = ColumnMetaData.scalar(Types.VARCHAR, "VARCHAR", Rep.STRING);
+      List<Array> varcharArrays = new ArrayList<>();
+      for (int i = 0; i < 5; i++) {
+        List<String> value = Collections.singletonList(Integer.toString(i));
+        varcharArrays.add(createArray("VARCHAR", varcharComponent, value));
+      }
+      writeAndReadArrays(conn, "varchar_arrays", "VARCHAR(30)",
+          varcharComponent, varcharArrays, PRIMITIVE_LIST_VALIDATOR);
+    }
+  }
+
+  @Test public void booleanArrays() throws Exception {
+    final Random r = new Random();
+    try (Connection conn = DriverManager.getConnection(url)) {
+      ScalarType component = ColumnMetaData.scalar(Types.BOOLEAN, "BOOLEAN", Rep.BOOLEAN);
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 5; i++) {
+        List<Boolean> elements = new ArrayList<>();
+        for (int j = 0; j < 5; j++) {
+          switch (r.nextInt(3)) {
+          case 0:
+            elements.add(Boolean.FALSE);
+            break;
+          case 1:
+            elements.add(Boolean.TRUE);
+            break;
+          case 2:
+            elements.add(null);
+            break;
+          default:
+            fail();
+          }
+        }
+        arrays.add(createArray("BOOLEAN", component, elements));
+      }
+      // Verify we can read and write the data
+      writeAndReadArrays(conn, "boolean_arrays", "BOOLEAN", component, arrays,
+          PRIMITIVE_LIST_VALIDATOR);
+    }
+  }
+
+  @Test public void shortArrays() throws Exception {
+    final Random r = new Random();
+    try (Connection conn = DriverManager.getConnection(url)) {
+      ScalarType component = ColumnMetaData.scalar(Types.SMALLINT, "SMALLINT", Rep.SHORT);
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 5; i++) {
+        List<Short> elements = new ArrayList<>();
+        for (int j = 0; j < 5; j++) {
+          short value = (short) r.nextInt(Short.MAX_VALUE);
+          // 50% of the time, negate the value
+          if (0 == r.nextInt(2)) {
+            value *= -1;
+          }
+          elements.add(Short.valueOf(value));
+        }
+        arrays.add(createArray("SMALLINT", component, elements));
+      }
+      // Verify read/write
+      writeAndReadArrays(conn, "short_arrays", "SMALLINT", component, arrays,
+          PRIMITIVE_LIST_VALIDATOR);
+    }
+  }
+
+  @Test public void shortArraysWithNull() throws Exception {
+    final Random r = new Random();
+    try (Connection conn = DriverManager.getConnection(url)) {
+      ScalarType component = ColumnMetaData.scalar(Types.SMALLINT, "SMALLINT", Rep.SHORT);
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 5; i++) {
+        List<Short> elements = new ArrayList<>();
+        for (int j = 0; j < 4; j++) {
+          short value = (short) r.nextInt(Short.MAX_VALUE);
+          // 50% of the time, negate the value
+          if (0 == r.nextInt(2)) {
+            value *= -1;
+          }
+          elements.add(Short.valueOf(value));
+        }
+        elements.add(null);
+        arrays.add(createArray("SMALLINT", component, elements));
+      }
+      // Verify read/write
+      writeAndReadArrays(conn, "short_arrays", "SMALLINT", component, arrays,
+          PRIMITIVE_LIST_VALIDATOR);
+    }
+  }
+
+  @Test public void longArrays() throws Exception {
+    final Random r = new Random();
+    try (Connection conn = DriverManager.getConnection(url)) {
+      ScalarType component = ColumnMetaData.scalar(Types.BIGINT, "BIGINT", Rep.LONG);
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 5; i++) {
+        List<Long> elements = new ArrayList<>();
+        for (int j = 0; j < 5; j++) {
+          elements.add(r.nextLong());
+        }
+        arrays.add(createArray("BIGINT", component, elements));
+      }
+      // Verify read/write
+      writeAndReadArrays(conn, "long_arrays", "BIGINT", component, arrays,
+          PRIMITIVE_LIST_VALIDATOR);
+    }
+  }
+
+  @Test public void stringArrays() throws Exception {
+    try (Connection conn = DriverManager.getConnection(url)) {
+      ScalarType component = ColumnMetaData.scalar(Types.VARCHAR, "VARCHAR", Rep.STRING);
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 5; i++) {
+        List<String> elements = new ArrayList<>();
+        for (int j = 0; j < 5; j++) {
+          elements.add(i + "_" + j);
+        }
+        arrays.add(createArray("VARCHAR", component, elements));
+      }
+      // Verify read/write
+      writeAndReadArrays(conn, "string_arrays", "VARCHAR", component, arrays,
+          PRIMITIVE_LIST_VALIDATOR);
+    }
+  }
+
+  @Test public void bigintArrays() throws Exception {
+    final Random r = new Random();
+    try (Connection conn = DriverManager.getConnection(url)) {
+      ScalarType component = ColumnMetaData.scalar(Types.BIGINT, "BIGINT", Rep.LONG);
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 3; i++) {
+        List<Long> elements = new ArrayList<>();
+        for (int j = 0; j < 7; j++) {
+          long element = r.nextLong();
+          if (r.nextBoolean()) {
+            element *= -1;
+          }
+          elements.add(element);
+        }
+        arrays.add(createArray("BIGINT", component, elements));
+      }
+      writeAndReadArrays(conn, "long_arrays", "BIGINT", component, arrays,
+          PRIMITIVE_LIST_VALIDATOR);
+    }
+  }
+
+  @Test public void doubleArrays() throws Exception {
+    final Random r = new Random();
+    try (Connection conn = DriverManager.getConnection(url)) {
+      ScalarType component = ColumnMetaData.scalar(Types.DOUBLE, "DOUBLE", Rep.DOUBLE);
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 3; i++) {
+        List<Double> elements = new ArrayList<>();
+        for (int j = 0; j < 7; j++) {
+          double element = r.nextDouble();
+          if (r.nextBoolean()) {
+            element *= -1;
+          }
+          elements.add(element);
+        }
+        arrays.add(createArray("DOUBLE", component, elements));
+      }
+      writeAndReadArrays(conn, "float_arrays", "DOUBLE", component, arrays,
+          PRIMITIVE_LIST_VALIDATOR);
+    }
+  }
+
+  @Test public void arraysOfByteArrays() throws Exception {
+    final Random r = new Random();
+    try (Connection conn = DriverManager.getConnection(url)) {
+      ScalarType component = ColumnMetaData.scalar(Types.TINYINT, "TINYINT", Rep.BYTE);
+      // [ Array([b, b, b]), Array([b, b, b]), ... ]
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 5; i++) {
+        List<Byte> elements = new ArrayList<>();
+        for (int j = 0; j < 5; j++) {
+          byte value = (byte) r.nextInt(Byte.MAX_VALUE);
+          // 50% of the time, negate the value
+          if (0 == r.nextInt(2)) {
+            value *= -1;
+          }
+          elements.add(Byte.valueOf(value));
+        }
+        arrays.add(createArray("TINYINT", component, elements));
+      }
+      // Verify read/write
+      writeAndReadArrays(conn, "byte_arrays", "TINYINT", component, arrays, BYTE_ARRAY_VALIDATOR);
+    }
+  }
+
+  @Test public void varbinaryArrays() throws Exception {
+    try (Connection conn = DriverManager.getConnection(url)) {
+      ScalarType component = ColumnMetaData.scalar(Types.VARBINARY, "VARBINARY", Rep.BYTE_STRING);
+      // [ Array(binary, binary, binary), Array(binary, binary, binary), ...]
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 5; i++) {
+        List<byte[]> elements = new ArrayList<>();
+        for (int j = 0; j < 5; j++) {
+          elements.add((i + "_" + j).getBytes(UTF_8));
+        }
+        arrays.add(createArray("VARBINARY", component, elements));
+      }
+      writeAndReadArrays(conn, "binary_arrays", "VARBINARY", component, arrays,
+          BYTE_ARRAY_ARRAY_VALIDATOR);
+    }
+  }
+
+  @Test public void timeArrays() throws Exception {
+    try (Connection conn = DriverManager.getConnection(url)) {
+      final long now = System.currentTimeMillis();
+      ScalarType component = ColumnMetaData.scalar(Types.TIME, "TIME", Rep.JAVA_SQL_TIME);
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 5; i++) {
+        List<Time> elements = new ArrayList<>();
+        for (int j = 0; j < 5; j++) {
+          elements.add(new Time(now + i + j));
+        }
+        arrays.add(createArray("TIME", component, elements));
+      }
+      writeAndReadArrays(conn, "time_arrays", "TIME", component, arrays, new Validator<Array>() {
+        @Override public void validate(Array expected, Array actual) throws SQLException {
+          Object[] expectedTimes = (Object[]) expected.getArray();
+          Object[] actualTimes = (Object[]) actual.getArray();
+          assertEquals(expectedTimes.length, actualTimes.length);
+          final Calendar cal = Unsafe.localCalendar();
+          for (int i = 0;  i < expectedTimes.length; i++) {
+            cal.setTime((Time) expectedTimes[i]);
+            int expectedHour = cal.get(Calendar.HOUR_OF_DAY);
+            int expectedMinute = cal.get(Calendar.MINUTE);
+            int expectedSecond = cal.get(Calendar.SECOND);
+            cal.setTime((Time) actualTimes[i]);
+            assertEquals(expectedHour, cal.get(Calendar.HOUR_OF_DAY));
+            assertEquals(expectedMinute, cal.get(Calendar.MINUTE));
+            assertEquals(expectedSecond, cal.get(Calendar.SECOND));
+          }
+        }
+      });
+      // Ensure an array with a null element can be written/read
+      Array arrayWithNull = createArray("TIME", component, Arrays.asList((Time) null));
+      writeAndReadArrays(conn, "time_array_with_null", "TIME", component,
+          Collections.singletonList(arrayWithNull), new Validator<Array>() {
+            @Override public void validate(Array expected, Array actual) throws Exception {
+              Object[] expectedArray = (Object[]) expected.getArray();
+              Object[] actualArray = (Object[]) actual.getArray();
+              assertEquals(1, expectedArray.length);
+              assertEquals(expectedArray.length, actualArray.length);
+              assertEquals(expectedArray[0], actualArray[0]);
+            }
+          });
+    }
+  }
+
+  @Test public void dateArrays() throws Exception {
+    try (Connection conn = DriverManager.getConnection(url)) {
+      final long now = System.currentTimeMillis();
+      ScalarType component = ColumnMetaData.scalar(Types.DATE, "DATE", Rep.JAVA_SQL_DATE);
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 5; i++) {
+        List<Date> elements = new ArrayList<>();
+        for (int j = 0; j < 5; j++) {
+          elements.add(new Date(now + i + j));
+        }
+        arrays.add(createArray("DATE", component, elements));
+      }
+      writeAndReadArrays(conn, "date_arrays", "DATE", component, arrays, new Validator<Array>() {
+        @Override public void validate(Array expected, Array actual) throws SQLException {
+          Object[] expectedDates = (Object[]) expected.getArray();
+          Object[] actualDates = (Object[]) actual.getArray();
+          assertEquals(expectedDates.length, actualDates.length);
+          final Calendar cal = Unsafe.localCalendar();
+          for (int i = 0;  i < expectedDates.length; i++) {
+            cal.setTime((Date) expectedDates[i]);
+            int expectedDayOfMonth = cal.get(Calendar.DAY_OF_MONTH);
+            int expectedMonth = cal.get(Calendar.MONTH);
+            int expectedYear = cal.get(Calendar.YEAR);
+            cal.setTime((Date) actualDates[i]);
+            assertEquals(expectedDayOfMonth, cal.get(Calendar.DAY_OF_MONTH));
+            assertEquals(expectedMonth, cal.get(Calendar.MONTH));
+            assertEquals(expectedYear, cal.get(Calendar.YEAR));
+          }
+        }
+      });
+      // Ensure an array with a null element can be written/read
+      Array arrayWithNull = createArray("DATE", component, Arrays.asList((Time) null));
+      writeAndReadArrays(conn, "date_array_with_null", "DATE", component,
+          Collections.singletonList(arrayWithNull), new Validator<Array>() {
+            @Override public void validate(Array expected, Array actual) throws Exception {
+              Object[] expectedArray = (Object[]) expected.getArray();
+              Object[] actualArray = (Object[]) actual.getArray();
+              assertEquals(1, expectedArray.length);
+              assertEquals(expectedArray.length, actualArray.length);
+              assertEquals(expectedArray[0], actualArray[0]);
+            }
+          });
+    }
+  }
+
+  @Test public void timestampArrays() throws Exception {
+    try (Connection conn = DriverManager.getConnection(url)) {
+      final long now = System.currentTimeMillis();
+      ScalarType component = ColumnMetaData.scalar(Types.TIMESTAMP, "TIMESTAMP",
+          Rep.JAVA_SQL_TIMESTAMP);
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 5; i++) {
+        List<Timestamp> elements = new ArrayList<>();
+        for (int j = 0; j < 5; j++) {
+          elements.add(new Timestamp(now + i + j));
+        }
+        arrays.add(createArray("TIMESTAMP", component, elements));
+      }
+      writeAndReadArrays(conn, "timestamp_arrays", "TIMESTAMP", component, arrays,
+          new Validator<Array>() {
+            @Override public void validate(Array expected, Array actual) throws SQLException {
+              Object[] expectedTimestamps = (Object[]) expected.getArray();
+              Object[] actualTimestamps = (Object[]) actual.getArray();
+              assertEquals(expectedTimestamps.length, actualTimestamps.length);
+              final Calendar cal = Unsafe.localCalendar();
+              for (int i = 0;  i < expectedTimestamps.length; i++) {
+                cal.setTime((Timestamp) expectedTimestamps[i]);
+                int expectedDayOfMonth = cal.get(Calendar.DAY_OF_MONTH);
+                int expectedMonth = cal.get(Calendar.MONTH);
+                int expectedYear = cal.get(Calendar.YEAR);
+                int expectedHour = cal.get(Calendar.HOUR_OF_DAY);
+                int expectedMinute = cal.get(Calendar.MINUTE);
+                int expectedSecond = cal.get(Calendar.SECOND);
+                int expectedMillisecond = cal.get(Calendar.MILLISECOND);
+                cal.setTime((Timestamp) actualTimestamps[i]);
+                assertEquals(expectedDayOfMonth, cal.get(Calendar.DAY_OF_MONTH));
+                assertEquals(expectedMonth, cal.get(Calendar.MONTH));
+                assertEquals(expectedYear, cal.get(Calendar.YEAR));
+                assertEquals(expectedHour, cal.get(Calendar.HOUR_OF_DAY));
+                assertEquals(expectedMinute, cal.get(Calendar.MINUTE));
+                assertEquals(expectedSecond, cal.get(Calendar.SECOND));
+                assertEquals(expectedMillisecond, cal.get(Calendar.MILLISECOND));
+              }
+            }
+          }
+      );
+      // Ensure an array with a null element can be written/read
+      Array arrayWithNull = createArray("TIMESTAMP", component, Arrays.asList((Timestamp) null));
+      writeAndReadArrays(conn, "timestamp_array_with_null", "TIMESTAMP", component,
+          Collections.singletonList(arrayWithNull), new Validator<Array>() {
+            @Override public void validate(Array expected, Array actual) throws Exception {
+              Object[] expectedArray = (Object[]) expected.getArray();
+              Object[] actualArray = (Object[]) actual.getArray();
+              assertEquals(1, expectedArray.length);
+              assertEquals(expectedArray.length, actualArray.length);
+              assertEquals(expectedArray[0], actualArray[0]);
+            }
+          });
+    }
+  }
+
+  @Test public void testCreateArrayOf() throws Exception {
+    try (Connection conn = DriverManager.getConnection(url)) {
+      final String componentName = SqlType.INTEGER.name();
+      Array a1 = conn.createArrayOf(componentName, new Object[] {1, 2, 3, 4, 5});
+      Array a2 = conn.createArrayOf(componentName, new Object[] {2, 3, 4, 5, 6});
+      Array a3 = conn.createArrayOf(componentName, new Object[] {3, 4, 5, 6, 7});
+      AvaticaType arrayType = ColumnMetaData.array(
+          ColumnMetaData.scalar(Types.INTEGER, componentName, Rep.INTEGER), "NUMBERS", Rep.ARRAY);
+      writeAndReadArrays(conn, "CREATE_ARRAY_OF_INTEGERS", componentName, arrayType,
+          Arrays.asList(a1, a2, a3), PRIMITIVE_LIST_VALIDATOR);
+    }
+  }
+
+  /**
+   * Creates a JDBC {@link Array} from a list of values.
+   *
+   * @param typeName the SQL type name of the elements in the array
+   * @param componentType The Avatica type for the array elements
+   * @param arrayValues The array elements
+   * @return An Array instance for the given component and values
+   */
+  @SuppressWarnings("unchecked")
+  private <T> Array createArray(String typeName, AvaticaType componentType, List<T> arrayValues) {
+    // Make a "row" with one "column" (which is really a list)
+    final List<Object> oneRow = Collections.singletonList((Object) arrayValues);
+    // Make an iterator over this one "row"
+    final Iterator<List<Object>> rowIterator = Collections.singletonList(oneRow).iterator();
+
+    ArrayType array = ColumnMetaData.array(componentType, typeName, Rep.ARRAY);
+    try (ListIteratorCursor cursor = new ListIteratorCursor(rowIterator)) {
+      List<ColumnMetaData> types = Collections.singletonList(ColumnMetaData.dummy(array, true));
+      Calendar calendar = Unsafe.localCalendar();
+      List<Accessor> accessors = cursor.createAccessors(types, calendar, null);
+      assertTrue("Expected at least one accessor, found " + accessors.size(),
+          !accessors.isEmpty());
+      ArrayAccessor arrayAccessor = (ArrayAccessor) accessors.get(0);
+
+      return new ArrayImpl((List<Object>) arrayValues, arrayAccessor);
+    }
+  }
+
+  /**
+   * Creates a table, writes the arrays to the table, and then verifies that the arrays can be
+   * read from that table and are equivalent to the original arrays.
+   *
+   * @param conn The JDBC connection
+   * @param tableName The name of the table to create and use
+   * @param componentType The component type of the array
+   * @param scalarType The Avatica type object for the component type of the array
+   * @param inputArrays The data to write and read
+   */
+  private void writeAndReadArrays(Connection conn, String tableName, String componentType,
+      AvaticaType scalarType, List<Array> inputArrays, Validator<Array> validator)
+      throws Exception {
+    // Drop and create the table
+    try (Statement stmt = conn.createStatement()) {
+      assertFalse(stmt.execute(Unsafe.formatLocalString("DROP TABLE IF EXISTS %s", tableName)));
+      String createTableSql = Unsafe.formatLocalString(
+          "CREATE TABLE %s (id integer, vals %s ARRAY)", tableName, componentType);
+      assertFalse(stmt.execute(createTableSql));
+    }
+
+    // Insert records, each with an array
+    final String dml = Unsafe.formatLocalString("INSERT INTO %s VALUES (?, ?)", tableName);
+    try (PreparedStatement stmt = conn.prepareStatement(dml)) {
+      int i = 0;
+      for (Array inputArray : inputArrays)  {
+        stmt.setInt(1, i);
+        stmt.setArray(2, inputArray);
+        assertEquals(1, stmt.executeUpdate());
+        i++;
+      }
+    }
+
+    // Read the records
+    try (Statement stmt = conn.createStatement()) {
+      ResultSet results = stmt.executeQuery(
+          Unsafe.formatLocalString("SELECT * FROM %s", tableName));
+      assertNotNull("Expected a ResultSet", results);
+      int i = 0;
+      for (Array expectedArray : inputArrays) {
+        assertTrue(results.next());
+        assertEquals(i++, results.getInt(1));
+        Array actualArray = results.getArray(2);
+
+        validator.validate(expectedArray, actualArray);
+
+        // TODO Fix this. See {@link AvaticaResultSet#create(ColumnMetaData.AvaticaType,Iterable)}
+        //ResultSet inputResults = expectedArray.getResultSet();
+        //ResultSet actualResult = actualArray.getResultSet();
+      }
+      assertFalse("Expected no more records", results.next());
+    }
+  }
+
+  /**
+   * A simple interface to validate to objects in support of type test cases
+   */
+  private interface Validator<T> {
+    void validate(T expected, T actual) throws Exception;
+  }
+
+  private static final PrimitiveArrayValidator PRIMITIVE_LIST_VALIDATOR =
+      new PrimitiveArrayValidator();
+  /**
+   * Validator that coerces primitive arrays into lists and comparse them.
+   */
+  private static class PrimitiveArrayValidator implements Validator<Array> {
+    @Override public void validate(Array expected, Array actual) throws SQLException {
+      assertEquals(AvaticaUtils.primitiveList(expected.getArray()),
+          AvaticaUtils.primitiveList(actual.getArray()));
+    }
+  }
+
+  private static final ByteArrayValidator BYTE_ARRAY_VALIDATOR = new ByteArrayValidator();
+  /**
+   * Validator that compares lists of bytes (the object).
+   */
+  private static class ByteArrayValidator implements Validator<Array> {
+    @SuppressWarnings("unchecked")
+    @Override public void validate(Array expected, Array actual) throws SQLException {
+      // Need to compare the byte arrays.
+      List<Byte> expectedArray =
+          (List<Byte>) AvaticaUtils.primitiveList(expected.getArray());
+      List<Byte> actualArray =
+          (List<Byte>) AvaticaUtils.primitiveList(actual.getArray());
+      assertEquals(expectedArray.size(), actualArray.size());
+
+      for (int j = 0; j < expectedArray.size(); j++) {
+        Byte expectedByte = expectedArray.get(j);
+        Byte actualByte = actualArray.get(j);
+        assertEquals(expectedByte, actualByte);
+      }
+    }
+  }
+
+  // Arrays of byte arrays (e.g. an Array<Varbinary>)
+  private static final ByteArrayArrayValidator BYTE_ARRAY_ARRAY_VALIDATOR =
+      new ByteArrayArrayValidator();
+  /**
+   * Validator that compares lists of byte arrays.
+   */
+  private static class ByteArrayArrayValidator implements Validator<Array> {
+    @SuppressWarnings("unchecked")
+    @Override public void validate(Array expected, Array actual) throws SQLException {
+      // Need to compare the byte arrays.
+      List<byte[]> expectedArray =
+          (List<byte[]>) AvaticaUtils.primitiveList(expected.getArray());
+      List<byte[]> actualArray =
+          (List<byte[]>) AvaticaUtils.primitiveList(actual.getArray());
+      assertEquals(expectedArray.size(), actualArray.size());
+
+      for (int j = 0; j < expectedArray.size(); j++) {
+        byte[] expectedBytes = expectedArray.get(j);
+        byte[] actualBytes = actualArray.get(j);
+        assertArrayEquals(expectedBytes, actualBytes);
+      }
+    }
+  }
+}
+
+// End ArrayTypeTest.java


[4/4] calcite-avatica git commit: [CALCITE-1756] Differentiate between implicitly null and explicitly null TypedValues

Posted by el...@apache.org.
[CALCITE-1756] Differentiate between implicitly null and explicitly null TypedValues

Noticed that a test should have been failing in the ARRAY work. The null
TypedValue was getting lost in the serialization. We need to handle
these two cases separately to ensure that the "intent" is preserved.


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

Branch: refs/heads/master
Commit: 557f15cc74ade0e4bd509838b263f85a7b3d0215
Parents: dd65a2b
Author: Josh Elser <el...@apache.org>
Authored: Mon Apr 17 11:40:55 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon Apr 24 17:59:24 2017 -0400

----------------------------------------------------------------------
 .../apache/calcite/avatica/proto/Common.java    | 185 ++++++++++++++-----
 .../apache/calcite/avatica/remote/Service.java  |   4 +-
 .../calcite/avatica/remote/TypedValue.java      |  20 +-
 core/src/main/protobuf/common.proto             |   3 +
 .../remote/ProtobufTranslationImplTest.java     |  12 ++
 5 files changed, 164 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/557f15cc/core/src/main/java/org/apache/calcite/avatica/proto/Common.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/proto/Common.java b/core/src/main/java/org/apache/calcite/avatica/proto/Common.java
index 50450f4..3bc10b5 100644
--- a/core/src/main/java/org/apache/calcite/avatica/proto/Common.java
+++ b/core/src/main/java/org/apache/calcite/avatica/proto/Common.java
@@ -14596,6 +14596,15 @@ public final class Common {
      * <code>optional .Rep component_type = 9;</code>
      */
     org.apache.calcite.avatica.proto.Common.Rep getComponentType();
+
+    /**
+     * <pre>
+     * Differentiate between explicitly null (user-set) and implicitly null
+     * </pre>
+     *
+     * <code>optional bool implicitly_null = 10;</code>
+     */
+    boolean getImplicitlyNull();
   }
   /**
    * <pre>
@@ -14622,6 +14631,7 @@ public final class Common {
       null_ = false;
       arrayValue_ = java.util.Collections.emptyList();
       componentType_ = 0;
+      implicitlyNull_ = false;
     }
 
     @java.lang.Override
@@ -14701,6 +14711,11 @@ public final class Common {
               componentType_ = rawValue;
               break;
             }
+            case 80: {
+
+              implicitlyNull_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -14938,6 +14953,19 @@ public final class Common {
       return result == null ? org.apache.calcite.avatica.proto.Common.Rep.UNRECOGNIZED : result;
     }
 
+    public static final int IMPLICITLY_NULL_FIELD_NUMBER = 10;
+    private boolean implicitlyNull_;
+    /**
+     * <pre>
+     * Differentiate between explicitly null (user-set) and implicitly null
+     * </pre>
+     *
+     * <code>optional bool implicitly_null = 10;</code>
+     */
+    public boolean getImplicitlyNull() {
+      return implicitlyNull_;
+    }
+
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
@@ -14977,6 +15005,9 @@ public final class Common {
       if (componentType_ != org.apache.calcite.avatica.proto.Common.Rep.PRIMITIVE_BOOLEAN.getNumber()) {
         output.writeEnum(9, componentType_);
       }
+      if (implicitlyNull_ != false) {
+        output.writeBool(10, implicitlyNull_);
+      }
     }
 
     public int getSerializedSize() {
@@ -15019,6 +15050,10 @@ public final class Common {
         size += com.google.protobuf.CodedOutputStream
           .computeEnumSize(9, componentType_);
       }
+      if (implicitlyNull_ != false) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(10, implicitlyNull_);
+      }
       memoizedSize = size;
       return size;
     }
@@ -15053,6 +15088,8 @@ public final class Common {
       result = result && getArrayValueList()
           .equals(other.getArrayValueList());
       result = result && componentType_ == other.componentType_;
+      result = result && (getImplicitlyNull()
+          == other.getImplicitlyNull());
       return result;
     }
 
@@ -15087,6 +15124,9 @@ public final class Common {
       }
       hash = (37 * hash) + COMPONENT_TYPE_FIELD_NUMBER;
       hash = (53 * hash) + componentType_;
+      hash = (37 * hash) + IMPLICITLY_NULL_FIELD_NUMBER;
+      hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
+          getImplicitlyNull());
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -15232,6 +15272,8 @@ public final class Common {
         }
         componentType_ = 0;
 
+        implicitlyNull_ = false;
+
         return this;
       }
 
@@ -15273,6 +15315,7 @@ public final class Common {
           result.arrayValue_ = arrayValueBuilder_.build();
         }
         result.componentType_ = componentType_;
+        result.implicitlyNull_ = implicitlyNull_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -15366,6 +15409,9 @@ public final class Common {
         if (other.componentType_ != 0) {
           setComponentTypeValue(other.getComponentTypeValue());
         }
+        if (other.getImplicitlyNull() != false) {
+          setImplicitlyNull(other.getImplicitlyNull());
+        }
         onChanged();
         return this;
       }
@@ -16114,6 +16160,44 @@ public final class Common {
         onChanged();
         return this;
       }
+
+      private boolean implicitlyNull_ ;
+      /**
+       * <pre>
+       * Differentiate between explicitly null (user-set) and implicitly null
+       * </pre>
+       *
+       * <code>optional bool implicitly_null = 10;</code>
+       */
+      public boolean getImplicitlyNull() {
+        return implicitlyNull_;
+      }
+      /**
+       * <pre>
+       * Differentiate between explicitly null (user-set) and implicitly null
+       * </pre>
+       *
+       * <code>optional bool implicitly_null = 10;</code>
+       */
+      public Builder setImplicitlyNull(boolean value) {
+        
+        implicitlyNull_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * Differentiate between explicitly null (user-set) and implicitly null
+       * </pre>
+       *
+       * <code>optional bool implicitly_null = 10;</code>
+       */
+      public Builder clearImplicitlyNull() {
+        
+        implicitlyNull_ = false;
+        onChanged();
+        return this;
+      }
       public final Builder setUnknownFields(
           final com.google.protobuf.UnknownFieldSet unknownFields) {
         return this;
@@ -18770,60 +18854,61 @@ public final class Common {
       " \001(\014\"\207\001\n\013ColumnValue\022\032\n\005value\030\001 \003(\0132\013.Ty",
       "pedValue\022 \n\013array_value\030\002 \003(\0132\013.TypedVal" +
       "ue\022\027\n\017has_array_value\030\003 \001(\010\022!\n\014scalar_va" +
-      "lue\030\004 \001(\0132\013.TypedValue\"\331\001\n\nTypedValue\022\022\n" +
+      "lue\030\004 \001(\0132\013.TypedValue\"\362\001\n\nTypedValue\022\022\n" +
       "\004type\030\001 \001(\0162\004.Rep\022\022\n\nbool_value\030\002 \001(\010\022\024\n" +
       "\014string_value\030\003 \001(\t\022\024\n\014number_value\030\004 \001(" +
       "\022\022\023\n\013bytes_value\030\005 \001(\014\022\024\n\014double_value\030\006" +
       " \001(\001\022\014\n\004null\030\007 \001(\010\022 \n\013array_value\030\010 \003(\0132" +
       "\013.TypedValue\022\034\n\016component_type\030\t \001(\0162\004.R" +
-      "ep\"\246\002\n\031MetaDataOperationArgument\022\024\n\014stri" +
-      "ng_value\030\001 \001(\t\022\022\n\nbool_value\030\002 \001(\010\022\021\n\tin",
-      "t_value\030\003 \001(\021\022\033\n\023string_array_values\030\004 \003" +
-      "(\t\022\030\n\020int_array_values\030\005 \003(\021\0225\n\004type\030\006 \001" +
-      "(\0162\'.MetaDataOperationArgument.ArgumentT" +
-      "ype\"^\n\014ArgumentType\022\n\n\006STRING\020\000\022\010\n\004BOOL\020" +
-      "\001\022\007\n\003INT\020\002\022\023\n\017REPEATED_STRING\020\003\022\020\n\014REPEA" +
-      "TED_INT\020\004\022\010\n\004NULL\020\005\"\260\001\n\nQueryState\022\030\n\004ty" +
-      "pe\030\001 \001(\0162\n.StateType\022\013\n\003sql\030\002 \001(\t\022\036\n\002op\030" +
-      "\003 \001(\0162\022.MetaDataOperation\022(\n\004args\030\004 \003(\0132" +
-      "\032.MetaDataOperationArgument\022\020\n\010has_args\030" +
-      "\005 \001(\010\022\017\n\007has_sql\030\006 \001(\010\022\016\n\006has_op\030\007 \001(\010*\237",
-      "\001\n\rStatementType\022\n\n\006SELECT\020\000\022\n\n\006INSERT\020\001" +
-      "\022\n\n\006UPDATE\020\002\022\n\n\006DELETE\020\003\022\n\n\006UPSERT\020\004\022\t\n\005" +
-      "MERGE\020\005\022\r\n\tOTHER_DML\020\006\022\n\n\006CREATE\020\007\022\010\n\004DR" +
-      "OP\020\010\022\t\n\005ALTER\020\t\022\r\n\tOTHER_DDL\020\n\022\010\n\004CALL\020\013" +
-      "*\342\003\n\003Rep\022\025\n\021PRIMITIVE_BOOLEAN\020\000\022\022\n\016PRIMI" +
-      "TIVE_BYTE\020\001\022\022\n\016PRIMITIVE_CHAR\020\002\022\023\n\017PRIMI" +
-      "TIVE_SHORT\020\003\022\021\n\rPRIMITIVE_INT\020\004\022\022\n\016PRIMI" +
-      "TIVE_LONG\020\005\022\023\n\017PRIMITIVE_FLOAT\020\006\022\024\n\020PRIM" +
-      "ITIVE_DOUBLE\020\007\022\013\n\007BOOLEAN\020\010\022\010\n\004BYTE\020\t\022\r\n" +
-      "\tCHARACTER\020\n\022\t\n\005SHORT\020\013\022\013\n\007INTEGER\020\014\022\010\n\004",
-      "LONG\020\r\022\t\n\005FLOAT\020\016\022\n\n\006DOUBLE\020\017\022\017\n\013BIG_INT" +
-      "EGER\020\031\022\017\n\013BIG_DECIMAL\020\032\022\021\n\rJAVA_SQL_TIME" +
-      "\020\020\022\026\n\022JAVA_SQL_TIMESTAMP\020\021\022\021\n\rJAVA_SQL_D" +
-      "ATE\020\022\022\022\n\016JAVA_UTIL_DATE\020\023\022\017\n\013BYTE_STRING" +
-      "\020\024\022\n\n\006STRING\020\025\022\n\n\006NUMBER\020\026\022\n\n\006OBJECT\020\027\022\010" +
-      "\n\004NULL\020\030\022\t\n\005ARRAY\020\033\022\n\n\006STRUCT\020\034\022\014\n\010MULTI" +
-      "SET\020\035*^\n\010Severity\022\024\n\020UNKNOWN_SEVERITY\020\000\022" +
-      "\022\n\016FATAL_SEVERITY\020\001\022\022\n\016ERROR_SEVERITY\020\002\022" +
-      "\024\n\020WARNING_SEVERITY\020\003*\327\004\n\021MetaDataOperat" +
-      "ion\022\022\n\016GET_ATTRIBUTES\020\000\022\033\n\027GET_BEST_ROW_",
-      "IDENTIFIER\020\001\022\020\n\014GET_CATALOGS\020\002\022\036\n\032GET_CL" +
-      "IENT_INFO_PROPERTIES\020\003\022\031\n\025GET_COLUMN_PRI" +
-      "VILEGES\020\004\022\017\n\013GET_COLUMNS\020\005\022\027\n\023GET_CROSS_" +
-      "REFERENCE\020\006\022\025\n\021GET_EXPORTED_KEYS\020\007\022\030\n\024GE" +
-      "T_FUNCTION_COLUMNS\020\010\022\021\n\rGET_FUNCTIONS\020\t\022" +
-      "\025\n\021GET_IMPORTED_KEYS\020\n\022\022\n\016GET_INDEX_INFO" +
-      "\020\013\022\024\n\020GET_PRIMARY_KEYS\020\014\022\031\n\025GET_PROCEDUR" +
-      "E_COLUMNS\020\r\022\022\n\016GET_PROCEDURES\020\016\022\026\n\022GET_P" +
-      "SEUDO_COLUMNS\020\017\022\017\n\013GET_SCHEMAS\020\020\022\031\n\025GET_" +
-      "SCHEMAS_WITH_ARGS\020\021\022\024\n\020GET_SUPER_TABLES\020",
-      "\022\022\023\n\017GET_SUPER_TYPES\020\023\022\030\n\024GET_TABLE_PRIV" +
-      "ILEGES\020\024\022\016\n\nGET_TABLES\020\025\022\023\n\017GET_TABLE_TY" +
-      "PES\020\026\022\021\n\rGET_TYPE_INFO\020\027\022\014\n\010GET_UDTS\020\030\022\027" +
-      "\n\023GET_VERSION_COLUMNS\020\031*\"\n\tStateType\022\007\n\003" +
-      "SQL\020\000\022\014\n\010METADATA\020\001B\"\n org.apache.calcit" +
-      "e.avatica.protob\006proto3"
+      "ep\022\027\n\017implicitly_null\030\n \001(\010\"\246\002\n\031MetaData" +
+      "OperationArgument\022\024\n\014string_value\030\001 \001(\t\022",
+      "\022\n\nbool_value\030\002 \001(\010\022\021\n\tint_value\030\003 \001(\021\022\033" +
+      "\n\023string_array_values\030\004 \003(\t\022\030\n\020int_array" +
+      "_values\030\005 \003(\021\0225\n\004type\030\006 \001(\0162\'.MetaDataOp" +
+      "erationArgument.ArgumentType\"^\n\014Argument" +
+      "Type\022\n\n\006STRING\020\000\022\010\n\004BOOL\020\001\022\007\n\003INT\020\002\022\023\n\017R" +
+      "EPEATED_STRING\020\003\022\020\n\014REPEATED_INT\020\004\022\010\n\004NU" +
+      "LL\020\005\"\260\001\n\nQueryState\022\030\n\004type\030\001 \001(\0162\n.Stat" +
+      "eType\022\013\n\003sql\030\002 \001(\t\022\036\n\002op\030\003 \001(\0162\022.MetaDat" +
+      "aOperation\022(\n\004args\030\004 \003(\0132\032.MetaDataOpera" +
+      "tionArgument\022\020\n\010has_args\030\005 \001(\010\022\017\n\007has_sq",
+      "l\030\006 \001(\010\022\016\n\006has_op\030\007 \001(\010*\237\001\n\rStatementTyp" +
+      "e\022\n\n\006SELECT\020\000\022\n\n\006INSERT\020\001\022\n\n\006UPDATE\020\002\022\n\n" +
+      "\006DELETE\020\003\022\n\n\006UPSERT\020\004\022\t\n\005MERGE\020\005\022\r\n\tOTHE" +
+      "R_DML\020\006\022\n\n\006CREATE\020\007\022\010\n\004DROP\020\010\022\t\n\005ALTER\020\t" +
+      "\022\r\n\tOTHER_DDL\020\n\022\010\n\004CALL\020\013*\342\003\n\003Rep\022\025\n\021PRI" +
+      "MITIVE_BOOLEAN\020\000\022\022\n\016PRIMITIVE_BYTE\020\001\022\022\n\016" +
+      "PRIMITIVE_CHAR\020\002\022\023\n\017PRIMITIVE_SHORT\020\003\022\021\n" +
+      "\rPRIMITIVE_INT\020\004\022\022\n\016PRIMITIVE_LONG\020\005\022\023\n\017" +
+      "PRIMITIVE_FLOAT\020\006\022\024\n\020PRIMITIVE_DOUBLE\020\007\022" +
+      "\013\n\007BOOLEAN\020\010\022\010\n\004BYTE\020\t\022\r\n\tCHARACTER\020\n\022\t\n",
+      "\005SHORT\020\013\022\013\n\007INTEGER\020\014\022\010\n\004LONG\020\r\022\t\n\005FLOAT" +
+      "\020\016\022\n\n\006DOUBLE\020\017\022\017\n\013BIG_INTEGER\020\031\022\017\n\013BIG_D" +
+      "ECIMAL\020\032\022\021\n\rJAVA_SQL_TIME\020\020\022\026\n\022JAVA_SQL_" +
+      "TIMESTAMP\020\021\022\021\n\rJAVA_SQL_DATE\020\022\022\022\n\016JAVA_U" +
+      "TIL_DATE\020\023\022\017\n\013BYTE_STRING\020\024\022\n\n\006STRING\020\025\022" +
+      "\n\n\006NUMBER\020\026\022\n\n\006OBJECT\020\027\022\010\n\004NULL\020\030\022\t\n\005ARR" +
+      "AY\020\033\022\n\n\006STRUCT\020\034\022\014\n\010MULTISET\020\035*^\n\010Severi" +
+      "ty\022\024\n\020UNKNOWN_SEVERITY\020\000\022\022\n\016FATAL_SEVERI" +
+      "TY\020\001\022\022\n\016ERROR_SEVERITY\020\002\022\024\n\020WARNING_SEVE" +
+      "RITY\020\003*\327\004\n\021MetaDataOperation\022\022\n\016GET_ATTR",
+      "IBUTES\020\000\022\033\n\027GET_BEST_ROW_IDENTIFIER\020\001\022\020\n" +
+      "\014GET_CATALOGS\020\002\022\036\n\032GET_CLIENT_INFO_PROPE" +
+      "RTIES\020\003\022\031\n\025GET_COLUMN_PRIVILEGES\020\004\022\017\n\013GE" +
+      "T_COLUMNS\020\005\022\027\n\023GET_CROSS_REFERENCE\020\006\022\025\n\021" +
+      "GET_EXPORTED_KEYS\020\007\022\030\n\024GET_FUNCTION_COLU" +
+      "MNS\020\010\022\021\n\rGET_FUNCTIONS\020\t\022\025\n\021GET_IMPORTED" +
+      "_KEYS\020\n\022\022\n\016GET_INDEX_INFO\020\013\022\024\n\020GET_PRIMA" +
+      "RY_KEYS\020\014\022\031\n\025GET_PROCEDURE_COLUMNS\020\r\022\022\n\016" +
+      "GET_PROCEDURES\020\016\022\026\n\022GET_PSEUDO_COLUMNS\020\017" +
+      "\022\017\n\013GET_SCHEMAS\020\020\022\031\n\025GET_SCHEMAS_WITH_AR",
+      "GS\020\021\022\024\n\020GET_SUPER_TABLES\020\022\022\023\n\017GET_SUPER_" +
+      "TYPES\020\023\022\030\n\024GET_TABLE_PRIVILEGES\020\024\022\016\n\nGET" +
+      "_TABLES\020\025\022\023\n\017GET_TABLE_TYPES\020\026\022\021\n\rGET_TY" +
+      "PE_INFO\020\027\022\014\n\010GET_UDTS\020\030\022\027\n\023GET_VERSION_C" +
+      "OLUMNS\020\031*\"\n\tStateType\022\007\n\003SQL\020\000\022\014\n\010METADA" +
+      "TA\020\001B\"\n org.apache.calcite.avatica.proto" +
+      "b\006proto3"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -18914,7 +18999,7 @@ public final class Common {
     internal_static_TypedValue_fieldAccessorTable = new
       com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_TypedValue_descriptor,
-        new java.lang.String[] { "Type", "BoolValue", "StringValue", "NumberValue", "BytesValue", "DoubleValue", "Null", "ArrayValue", "ComponentType", });
+        new java.lang.String[] { "Type", "BoolValue", "StringValue", "NumberValue", "BytesValue", "DoubleValue", "Null", "ArrayValue", "ComponentType", "ImplicitlyNull", });
     internal_static_MetaDataOperationArgument_descriptor =
       getDescriptor().getMessageTypes().get(13);
     internal_static_MetaDataOperationArgument_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/557f15cc/core/src/main/java/org/apache/calcite/avatica/remote/Service.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/Service.java b/core/src/main/java/org/apache/calcite/avatica/remote/Service.java
index 8178ecd..b935b22 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/Service.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/Service.java
@@ -1045,7 +1045,7 @@ public interface Service {
       if (msg.getHasParameterValues()) {
         values = new ArrayList<>(msg.getParameterValuesCount());
         for (Common.TypedValue valueProto : msg.getParameterValuesList()) {
-          if (TypedValue.NULL_PROTO.equals(valueProto)) {
+          if (TypedValue.PROTO_IMPLICIT_NULL.equals(valueProto)) {
             values.add(null);
           } else {
             values.add(TypedValue.fromProto(valueProto));
@@ -1076,7 +1076,7 @@ public interface Service {
         builder.setHasParameterValues(true);
         for (TypedValue paramValue : parameterValues) {
           if (paramValue == null) {
-            builder.addParameterValues(TypedValue.NULL_PROTO);
+            builder.addParameterValues(TypedValue.PROTO_IMPLICIT_NULL);
           } else {
             builder.addParameterValues(paramValue.toProto());
           }

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/557f15cc/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java b/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
index 04268bb..363825b 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/TypedValue.java
@@ -145,9 +145,12 @@ public class TypedValue {
   private static final FieldDescriptor BYTES_DESCRIPTOR = Common.TypedValue.getDescriptor()
       .findFieldByNumber(Common.TypedValue.BYTES_VALUE_FIELD_NUMBER);
 
-  public static final TypedValue NULL =
+  // If the user sets a `null` Object, it's explicitly null
+  public static final TypedValue EXPLICIT_NULL =
       new TypedValue(ColumnMetaData.Rep.OBJECT, null);
-  public static final Common.TypedValue NULL_PROTO = NULL.toProto();
+  // The user might also implicitly not set a value for a parameter.
+  public static final Common.TypedValue PROTO_IMPLICIT_NULL =
+      Common.TypedValue.newBuilder().setImplicitlyNull(true).build();
 
   /** Type of the value. */
   public final ColumnMetaData.Rep type;
@@ -194,7 +197,7 @@ public class TypedValue {
   public static TypedValue create(@JsonProperty("type") String type,
       @JsonProperty("value") Object value) {
     if (value == null) {
-      return NULL;
+      return EXPLICIT_NULL;
     }
     ColumnMetaData.Rep rep = ColumnMetaData.Rep.valueOf(type);
     return ofLocal(rep, serialToLocal(rep, value));
@@ -214,7 +217,7 @@ public class TypedValue {
   public static TypedValue ofJdbc(ColumnMetaData.Rep rep, Object value,
       Calendar calendar) {
     if (value == null) {
-      return NULL;
+      return EXPLICIT_NULL;
     }
     final Object serialValue;
     if (ColumnMetaData.Rep.ARRAY == rep) {
@@ -243,7 +246,7 @@ public class TypedValue {
    * deducing its type. */
   public static TypedValue ofJdbc(Object value, Calendar calendar) {
     if (value == null) {
-      return NULL;
+      return EXPLICIT_NULL;
     }
     final ColumnMetaData.Rep rep = ColumnMetaData.Rep.of(value.getClass());
     return new TypedValue(rep, jdbcToSerial(rep, value, calendar));
@@ -485,6 +488,9 @@ public class TypedValue {
    */
   public Common.TypedValue toProto() {
     final Common.TypedValue.Builder builder = Common.TypedValue.newBuilder();
+    // This isn't a static method, therefore we have a non-null TypedValue. Thus, this message
+    // cannot be implicitly null
+    builder.setImplicitlyNull(false);
 
     Common.Rep protoRep = type.toProto();
     // Protobuf has an explicit BIG_DECIMAL representation enum value.
@@ -896,9 +902,7 @@ public class TypedValue {
       }
 
       if (null == value) {
-        if (null != other.value) {
-          return false;
-        }
+        return null == other.value;
       }
 
       return value.equals(other.value);

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/557f15cc/core/src/main/protobuf/common.proto
----------------------------------------------------------------------
diff --git a/core/src/main/protobuf/common.proto b/core/src/main/protobuf/common.proto
index 63dbcc9..04d0a85 100644
--- a/core/src/main/protobuf/common.proto
+++ b/core/src/main/protobuf/common.proto
@@ -202,6 +202,9 @@ message TypedValue {
 
   repeated TypedValue array_value = 8; // The Array
   Rep component_type = 9; // If an Array, the representation for the array values
+
+  bool implicitly_null = 10; // Differentiate between explicitly null (user-set) and implicitly null
+                            // (un-set by the user)
 }
 
 // The severity of some unexpected outcome to an operation.

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/557f15cc/core/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java b/core/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
index ebb2df4..8069c84 100644
--- a/core/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
+++ b/core/src/test/java/org/apache/calcite/avatica/remote/ProtobufTranslationImplTest.java
@@ -45,6 +45,7 @@ import org.apache.calcite.avatica.remote.Service.DatabasePropertyRequest;
 import org.apache.calcite.avatica.remote.Service.DatabasePropertyResponse;
 import org.apache.calcite.avatica.remote.Service.ErrorResponse;
 import org.apache.calcite.avatica.remote.Service.ExecuteBatchResponse;
+import org.apache.calcite.avatica.remote.Service.ExecuteRequest;
 import org.apache.calcite.avatica.remote.Service.ExecuteResponse;
 import org.apache.calcite.avatica.remote.Service.FetchRequest;
 import org.apache.calcite.avatica.remote.Service.FetchResponse;
@@ -223,6 +224,17 @@ public class ProtobufTranslationImplTest<T> {
     List<String> commands = Arrays.asList("command1", "command2", "command3");
     requests.add(new PrepareAndExecuteBatchRequest("connectionId", 12345, commands));
 
+
+    List<ColumnMetaData> columns = Collections.emptyList();
+    List<AvaticaParameter> params = Collections.emptyList();
+    Meta.CursorFactory cursorFactory = Meta.CursorFactory.create(Style.LIST, Object.class,
+        Collections.<String>emptyList());
+    Signature signature = Signature.create(columns, "sql", params, cursorFactory,
+        Meta.StatementType.SELECT);
+    Meta.StatementHandle handle = new Meta.StatementHandle("1234", 1, signature);
+    requests.add(new ExecuteRequest(handle, Arrays.<TypedValue>asList((TypedValue) null), 10));
+    requests.add(new ExecuteRequest(handle, Arrays.asList(TypedValue.EXPLICIT_NULL), 10));
+
     return requests;
   }