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:56 UTC

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

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