You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by za...@apache.org on 2021/02/26 18:38:47 UTC

[calcite-avatica] 02/03: Improve test coverage for float/real/double array types (Alessandro Solimando)

This is an automated email from the ASF dual-hosted git repository.

zabetak pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite-avatica.git

commit efbf8a61e13277d4cbb292716d1473b81ebb61a7
Author: Alessandro Solimando <18...@users.noreply.github.com>
AuthorDate: Sat Feb 20 20:24:39 2021 +0100

    Improve test coverage for float/real/double array types (Alessandro Solimando)
    
    1. Add missing unit tests for CALCITE-3163
    2. Reduce warnings in ArrayTypeTest.java
    3. Refactor related code to improve readability
    
    Close apache/calcite-avatica#139
---
 .../calcite/avatica/util/ArrayAccessorTest.java    | 145 +++++++++++++
 .../apache/calcite/avatica/util/ArrayImplTest.java | 134 +++++++-----
 .../calcite/avatica/util/AssertTestUtils.java      |  76 +++++++
 .../calcite/avatica/util/CursorTestUtils.java      |  65 ++++++
 .../calcite/avatica/remote/ArrayTypeTest.java      | 231 ++++++++++++---------
 5 files changed, 499 insertions(+), 152 deletions(-)

diff --git a/core/src/test/java/org/apache/calcite/avatica/util/ArrayAccessorTest.java b/core/src/test/java/org/apache/calcite/avatica/util/ArrayAccessorTest.java
new file mode 100644
index 0000000..a9efc93
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/avatica/util/ArrayAccessorTest.java
@@ -0,0 +1,145 @@
+/*
+ * 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.Rep;
+
+import org.apache.calcite.avatica.MetaImpl;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test class for verifying functionality in array accessor from abstract cursor.
+ */
+@RunWith(Parameterized.class)
+public class ArrayAccessorTest {
+
+  private static final double DELTA = 1e-15;
+  private static final ArrayImpl.Factory ARRAY_FACTORY =
+      new ArrayFactoryImpl(Unsafe.localCalendar().getTimeZone());
+
+  private static final List<Function<List<List<Object>>, Cursor>> CURSOR_BUILDER =
+      Arrays.asList(CursorTestUtils::createArrayBasedCursor,
+          CursorTestUtils::createListBasedCursor);
+
+  private Function<List<List<Object>>, Cursor> cursorBuilder;
+
+  @Parameterized.Parameters
+  public static List<Object[]> parameters() throws Exception {
+    return CURSOR_BUILDER.stream()
+        .map(Collections::singletonList)
+        .map(List::toArray)
+        .collect(Collectors.toList());
+  }
+
+  public ArrayAccessorTest(Function<List<List<Object>>, Cursor> cursorBuilder) {
+    this.cursorBuilder = cursorBuilder;
+  }
+
+  @Test public void listIteratorFromIntegerArray() throws Exception {
+    AssertTestUtils.Validator validator =
+        (Object o1, Object o2) -> assertEquals((int) o1, (int) o2);
+
+    ColumnMetaData.ScalarType intType = ColumnMetaData.scalar(Types.INTEGER, "INTEGER", Rep.INTEGER);
+
+    ColumnMetaData arrayMetadata = createArrayMetaData(intType);
+
+    List<List<Object>> rowsValues = Arrays.asList(Arrays.asList(1, 2),
+        Collections.singletonList(3), Arrays.asList(4, 5, 6));
+
+    try (Cursor cursor = cursorBuilder.apply(rowsValues)) {
+      AssertTestUtils.assertRowsValuesMatchCursorContentViaArrayAccessor(
+          rowsValues, intType, cursor, arrayMetadata, ARRAY_FACTORY, validator);
+    }
+  }
+
+  @Test public void resultSetFromRealArray() throws Exception {
+    AssertTestUtils.Validator validator =
+        (Object o1, Object o2) -> assertEquals((float) o1, (float) o2, DELTA);
+
+    ColumnMetaData.ScalarType realType = ColumnMetaData.scalar(Types.REAL, "REAL", Rep.FLOAT);
+
+    ColumnMetaData arrayMetadata = createArrayMetaData(realType);
+
+    List<List<Object>> rowsValues = Arrays.asList(
+        Arrays.asList(1.123f, 0.2f),
+        Arrays.asList(4.1f, 5f, 66.12345f)
+    );
+
+    try (Cursor cursor = cursorBuilder.apply(rowsValues)) {
+      AssertTestUtils.assertRowsValuesMatchCursorContentViaArrayAccessor(
+          rowsValues, realType, cursor, arrayMetadata, ARRAY_FACTORY, validator);
+    }
+  }
+
+  @Test public void resultSetFromDoubleArray() throws Exception {
+    AssertTestUtils.Validator validator =
+        (Object o1, Object o2) -> assertEquals((double) o1, (double) o2, DELTA);
+
+    ColumnMetaData.ScalarType doubleType = ColumnMetaData.scalar(Types.DOUBLE, "DOUBLE", Rep.DOUBLE);
+
+    ColumnMetaData arrayMetadata = createArrayMetaData(doubleType);
+
+    List<List<Object>> rowsValues = Arrays.asList(
+        Arrays.asList(1.123d, 0.123456789012d),
+        Arrays.asList(4.134555d, 54444d, 66.12345d)
+    );
+
+    try (Cursor cursor = cursorBuilder.apply(rowsValues)) {
+      AssertTestUtils.assertRowsValuesMatchCursorContentViaArrayAccessor(
+          rowsValues, doubleType, cursor, arrayMetadata, ARRAY_FACTORY, validator);
+    }
+  }
+
+  @Test public void resultSetFromFloatArray() throws Exception {
+    AssertTestUtils.Validator validator =
+        (Object o1, Object o2) -> assertEquals((double) o1, (double) o2, DELTA);
+
+    ColumnMetaData.ScalarType floatType = ColumnMetaData.scalar(Types.FLOAT, "FLOAT", Rep.DOUBLE);
+
+    ColumnMetaData arrayMetadata = createArrayMetaData(floatType);
+
+    List<List<Object>> rowsValues = Arrays.asList(
+        Arrays.asList(1.123d, 0.123456789012d),
+        Arrays.asList(4.134555d, 54444d, 66.12345d)
+    );
+
+    try (Cursor cursor = cursorBuilder.apply(rowsValues)) {
+      AssertTestUtils.assertRowsValuesMatchCursorContentViaArrayAccessor(
+          rowsValues, floatType, cursor, arrayMetadata, ARRAY_FACTORY, validator);
+    }
+  }
+
+  private static ColumnMetaData createArrayMetaData(ColumnMetaData.ScalarType componentType) {
+    ColumnMetaData.ArrayType arrayType =
+        ColumnMetaData.array(componentType, componentType.name, componentType.rep);
+    return MetaImpl.columnMetaData("MY_ARRAY", 1, arrayType, false);
+  }
+}
+
+// End ArrayAccessorTest.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
index 9133270..e2e4d9e 100644
--- a/core/src/test/java/org/apache/calcite/avatica/util/ArrayImplTest.java
+++ b/core/src/test/java/org/apache/calcite/avatica/util/ArrayImplTest.java
@@ -27,7 +27,6 @@ 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;
@@ -36,7 +35,6 @@ import java.util.List;
 import java.util.Objects;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -44,65 +42,81 @@ import static org.junit.Assert.assertTrue;
  */
 public class ArrayImplTest {
 
-  @Test public void resultSetFromArray() throws Exception {
-    // Define the struct type we're creating
+  private static final double DELTA = 1e-15;
+  private static final ArrayImpl.Factory ARRAY_FACTORY =
+      new ArrayFactoryImpl(Unsafe.localCalendar().getTimeZone());
+
+  @Test public void resultSetFromIntegerArray() throws Exception {
+    AssertTestUtils.Validator validator =
+        (Object o1, Object o2) -> assertEquals((int) o1, (int) o2);
+
     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);
+    ColumnMetaData arrayMetadata = createArrayMetaData(intType);
 
-      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());
+    List<List<Object>> rowsValues = Arrays.asList(Arrays.asList(1, 2),
+        Collections.singletonList(3), Arrays.asList(4, 5, 6));
 
-      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());
+    try (Cursor cursor =
+             CursorTestUtils.createArrayImplBasedCursor(rowsValues, intType, ARRAY_FACTORY)) {
+      AssertTestUtils.assertRowsValuesMatchCursorContentViaArrayAccessor(
+          rowsValues, intType, cursor, arrayMetadata, ARRAY_FACTORY, validator);
+    }
+  }
 
-      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 resultSetFromRealArray() throws Exception {
+    AssertTestUtils.Validator validator =
+        (Object o1, Object o2) -> assertEquals((float) o1, (float) o2, DELTA);
+
+    ScalarType realType = ColumnMetaData.scalar(Types.REAL, "REAL", Rep.FLOAT);
+    ColumnMetaData arrayMetadata = createArrayMetaData(realType);
+
+    List<List<Object>> rowsValues = Arrays.asList(
+        Arrays.asList(1.123f, 0.2f),
+        Arrays.asList(4.1f, 5f, 66.12345f)
+    );
+
+    try (Cursor cursor =
+             CursorTestUtils.createArrayImplBasedCursor(rowsValues, realType, ARRAY_FACTORY)) {
+      AssertTestUtils.assertRowsValuesMatchCursorContentViaArrayAccessor(
+          rowsValues, realType, cursor, arrayMetadata, ARRAY_FACTORY, validator);
+    }
+  }
+
+  @Test public void resultSetFromDoubleArray() throws Exception {
+    AssertTestUtils.Validator validator =
+        (Object o1, Object o2) -> assertEquals((double) o1, (double) o2, DELTA);
+
+    ScalarType doubleType = ColumnMetaData.scalar(Types.DOUBLE, "DOUBLE", Rep.PRIMITIVE_DOUBLE);
+    ColumnMetaData arrayMetadata = createArrayMetaData(doubleType);
+
+    List<List<Object>> rowsValues = Arrays.asList(
+        Arrays.asList(1.123d, 0.123456789012d),
+        Arrays.asList(4.134555d, 54444d, 66.12345d)
+    );
+
+    try (Cursor cursor =
+             CursorTestUtils.createArrayImplBasedCursor(rowsValues, doubleType, ARRAY_FACTORY)) {
+      AssertTestUtils.assertRowsValuesMatchCursorContentViaArrayAccessor(
+          rowsValues, doubleType, cursor, arrayMetadata, ARRAY_FACTORY, validator);
+    }
+  }
+
+  @Test public void resultSetFromFloatArray() throws Exception {
+    AssertTestUtils.Validator validator =
+        (Object o1, Object o2) -> assertEquals((double) o1, (double) o2, DELTA);
+
+    ScalarType floatType = ColumnMetaData.scalar(Types.FLOAT, "FLOAT", Rep.PRIMITIVE_DOUBLE);
+    ColumnMetaData arrayMetadata = createArrayMetaData(floatType);
+
+    List<List<Object>> rowsValues = Arrays.asList(
+        Arrays.asList(1.123d, 0.123456789012d),
+        Arrays.asList(4.134555d, 54444d, 66.12345d)
+    );
+
+    try (Cursor cursor =
+             CursorTestUtils.createArrayImplBasedCursor(rowsValues, floatType, ARRAY_FACTORY)) {
+      AssertTestUtils.assertRowsValuesMatchCursorContentViaArrayAccessor(
+          rowsValues, floatType, cursor, arrayMetadata, ARRAY_FACTORY, validator);
     }
   }
 
@@ -402,6 +416,12 @@ public class ArrayImplTest {
     assertEquals(5, data[1]);
     assertEquals(6, data[2]);
   }
+
+  private static ColumnMetaData createArrayMetaData(ColumnMetaData.ScalarType componentType) {
+    ColumnMetaData.ArrayType arrayType =
+        ColumnMetaData.array(componentType, componentType.name, componentType.rep);
+    return MetaImpl.columnMetaData("MY_ARRAY", 1, arrayType, false);
+  }
 }
 
 // End ArrayImplTest.java
diff --git a/core/src/test/java/org/apache/calcite/avatica/util/AssertTestUtils.java b/core/src/test/java/org/apache/calcite/avatica/util/AssertTestUtils.java
new file mode 100644
index 0000000..ac2ba60
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/avatica/util/AssertTestUtils.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.avatica.util;
+
+import org.apache.calcite.avatica.ColumnMetaData;
+
+import java.sql.Array;
+import java.sql.ResultSet;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Utilities for assertions in tests of the util package.
+ */
+public class AssertTestUtils {
+
+  private AssertTestUtils() {
+    // private constructor
+  }
+
+  /**
+   * A simple interface to validate expected and actual values.
+   */
+  interface Validator {
+    void validate(Object expected, Object actual);
+  }
+
+  static void assertRowsValuesMatchCursorContentViaArrayAccessor(
+      List<List<Object>> rowsValues, ColumnMetaData.ScalarType arrayContentMetadata,
+      Cursor cursorOverArray, ColumnMetaData arrayMetaData, ArrayImpl.Factory factory,
+      Validator validator) throws Exception {
+    List<Cursor.Accessor> accessors = cursorOverArray.createAccessors(
+        Collections.singletonList(arrayMetaData), Unsafe.localCalendar(), factory);
+    assertEquals(1, accessors.size());
+    Cursor.Accessor accessor = accessors.get(0);
+
+    for (List<Object> rowValue : rowsValues) {
+      assertTrue(cursorOverArray.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());
+      assertEquals(arrayContentMetadata.id, actualArrayResultSet.getMetaData().getColumnType(2));
+      assertTrue(actualArrayResultSet.next());
+
+      for (int j = 0; j < rowValue.size(); ++j) {
+        assertEquals(j + 1, actualArrayResultSet.getInt(1));
+        // ResultSet.getObject() uses the column type internally, we can rely on that
+        validator.validate(rowValue.get(j), actualArrayResultSet.getObject(2));
+        assertEquals(j < rowValue.size() - 1, actualArrayResultSet.next());
+      }
+    }
+    assertFalse(cursorOverArray.next());
+  }
+}
+
+// End AssertTestUtils.java
diff --git a/core/src/test/java/org/apache/calcite/avatica/util/CursorTestUtils.java b/core/src/test/java/org/apache/calcite/avatica/util/CursorTestUtils.java
new file mode 100644
index 0000000..91278fb
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/avatica/util/CursorTestUtils.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.avatica.util;
+
+import org.apache.calcite.avatica.ColumnMetaData;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Utilities methods for creating cursors from data for tests of the util package.
+ */
+public class CursorTestUtils {
+
+  private CursorTestUtils() {
+    // private constructor
+  }
+
+  static ListIteratorCursor createListBasedCursor(List<List<Object>> rowsValues) {
+    Iterator<List<Object>> iterator = rowsValues.stream()
+        .map(a -> (Object) a)
+        .map(Collections::singletonList)
+        .collect(Collectors.toList())
+        .iterator();
+    return new ListIteratorCursor(iterator);
+  }
+
+  static ArrayIteratorCursor createArrayBasedCursor(List<List<Object>> rowsValues) {
+    Iterator<Object[]> iterator = Arrays.stream(rowsValues.toArray())
+        .map(x -> Collections.singletonList(x).toArray())
+        .iterator();
+    return new ArrayIteratorCursor(iterator);
+  }
+
+  static ListIteratorCursor createArrayImplBasedCursor(
+      List<List<Object>> rowsValues, ColumnMetaData.ScalarType arrayComponentType,
+      ArrayImpl.Factory factory) {
+    Iterator<List<Object>> iterator = rowsValues.stream()
+        .map(vals -> factory.createArray(arrayComponentType, vals))
+        .map(a -> (Object) a)
+        .map(Collections::singletonList)
+        .collect(Collectors.toList())
+        .iterator();
+    return new ListIteratorCursor(iterator);
+  }
+}
+
+// End CursorTestUtils.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
index d73648d..d479a0a 100644
--- a/server/src/test/java/org/apache/calcite/avatica/remote/ArrayTypeTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/remote/ArrayTypeTest.java
@@ -92,9 +92,7 @@ public class ArrayTypeTest {
   }
 
   @AfterClass public static void afterClass() throws Exception {
-    if (null != SERVERS) {
-      SERVERS.stopServers();
-    }
+    SERVERS.stopServers();
   }
 
   @Test public void simpleArrayTest() throws Exception {
@@ -155,7 +153,7 @@ public class ArrayTypeTest {
           if (0 == r.nextInt(2)) {
             value *= -1;
           }
-          elements.add(Short.valueOf(value));
+          elements.add(value);
         }
         arrays.add(createArray("SMALLINT", component, elements));
       }
@@ -179,7 +177,7 @@ public class ArrayTypeTest {
           if (0 == r.nextInt(2)) {
             value *= -1;
           }
-          elements.add(Short.valueOf(value));
+          elements.add(value);
         }
         elements.add(null);
         arrays.add(createArray("SMALLINT", component, elements));
@@ -266,7 +264,59 @@ public class ArrayTypeTest {
         }
         arrays.add(createArray("DOUBLE", component, elements));
       }
-      writeAndReadArrays(conn, "float_arrays", "DOUBLE", component, arrays,
+      writeAndReadArrays(conn, "double_arrays", "DOUBLE", component, arrays,
+          PRIMITIVE_LIST_VALIDATOR);
+    }
+  }
+
+  @Test public void realArrays() throws Exception {
+    final Random r = new Random();
+    try (Connection conn = DriverManager.getConnection(url)) {
+      ScalarType component = ColumnMetaData.scalar(Types.REAL, "REAL", Rep.FLOAT);
+      List<Array> arrays = new ArrayList<>();
+      // Construct the data
+      for (int i = 0; i < 3; i++) {
+        List<Float> elements = new ArrayList<>();
+        for (int j = 0; j < 7; j++) {
+          float element = r.nextFloat();
+          if (r.nextBoolean()) {
+            element *= -1;
+          }
+          elements.add(element);
+        }
+        arrays.add(createArray("REAL", component, elements));
+      }
+      writeAndReadArrays(conn, "real_arrays", "REAL", component, arrays,
+          (expected, actual) -> {
+            // 'Real' maps to 'Float' following the JDBC specs, but hsqldb maps 'Double', 'Real'
+            // and 'Float' to Java 'double'
+            double[] expectedArray = Arrays.stream((Object[]) expected.getArray())
+                .mapToDouble(x -> ((Float) x).doubleValue()).toArray();
+            double[] actualArray = Arrays.stream((Object[]) actual.getArray())
+                .mapToDouble(x -> (double) x).toArray();
+            assertArrayEquals(expectedArray, actualArray, Double.MIN_VALUE);
+          });
+    }
+  }
+
+  @Test public void floatArrays() throws Exception {
+    final Random r = new Random();
+    try (Connection conn = DriverManager.getConnection(url)) {
+      ScalarType component = ColumnMetaData.scalar(Types.FLOAT, "FLOAT", Rep.FLOAT);
+      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.nextFloat();
+          if (r.nextBoolean()) {
+            element *= -1;
+          }
+          elements.add(element);
+        }
+        arrays.add(createArray("FLOAT", component, elements));
+      }
+      writeAndReadArrays(conn, "float_arrays", "FLOAT", component, arrays,
           PRIMITIVE_LIST_VALIDATOR);
     }
   }
@@ -286,12 +336,13 @@ public class ArrayTypeTest {
           if (0 == r.nextInt(2)) {
             value *= -1;
           }
-          elements.add(Byte.valueOf(value));
+          elements.add(value);
         }
         arrays.add(createArray("TINYINT", component, elements));
       }
       // Verify read/write
-      writeAndReadArrays(conn, "byte_arrays", "TINYINT", component, arrays, BYTE_ARRAY_VALIDATOR);
+      writeAndReadArrays(conn, "byte_arrays", "TINYINT", component, arrays,
+          BYTE_ARRAY_VALIDATOR);
     }
   }
 
@@ -326,35 +377,32 @@ public class ArrayTypeTest {
         }
         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));
-          }
-        }
-      });
+      writeAndReadArrays(conn, "time_arrays", "TIME", component, arrays,
+          (expected, actual) -> {
+            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]);
-            }
+          Collections.singletonList(arrayWithNull), (expected, actual) -> {
+            Object[] expectedArray = (Object[]) expected.getArray();
+            Object[] actualArray = (Object[]) actual.getArray();
+            assertEquals(1, expectedArray.length);
+            assertEquals(expectedArray.length, actualArray.length);
+            assertEquals(expectedArray[0], actualArray[0]);
           });
     }
   }
@@ -372,35 +420,32 @@ public class ArrayTypeTest {
         }
         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));
-          }
-        }
-      });
+      writeAndReadArrays(conn, "date_arrays", "DATE", component, arrays,
+          (expected, actual) -> {
+            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]);
-            }
+          Collections.singletonList(arrayWithNull), (expected, actual) -> {
+            Object[] expectedArray = (Object[]) expected.getArray();
+            Object[] actualArray = (Object[]) actual.getArray();
+            assertEquals(1, expectedArray.length);
+            assertEquals(expectedArray.length, actualArray.length);
+            assertEquals(expectedArray[0], actualArray[0]);
           });
     }
   }
@@ -420,43 +465,39 @@ public class ArrayTypeTest {
         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));
-              }
+          (expected, actual) -> {
+            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]);
-            }
+          Collections.singletonList(arrayWithNull), (expected, actual) -> {
+            Object[] expectedArray = (Object[]) expected.getArray();
+            Object[] actualArray = (Object[]) actual.getArray();
+            assertEquals(1, expectedArray.length);
+            assertEquals(expectedArray.length, actualArray.length);
+            assertEquals(expectedArray[0], actualArray[0]);
           });
     }
   }
@@ -547,8 +588,8 @@ public class ArrayTypeTest {
       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());
+      assertFalse("Expected at least one accessor, found " + accessors.size(),
+          accessors.isEmpty());
       ArrayAccessor arrayAccessor = (ArrayAccessor) accessors.get(0);
 
       return new ArrayImpl((List<Object>) arrayValues, arrayAccessor);
@@ -620,7 +661,7 @@ public class ArrayTypeTest {
   private static final PrimitiveArrayValidator PRIMITIVE_LIST_VALIDATOR =
       new PrimitiveArrayValidator();
   /**
-   * Validator that coerces primitive arrays into lists and comparse them.
+   * Validator that coerces primitive arrays into lists and compare them.
    */
   private static class PrimitiveArrayValidator implements Validator<Array> {
     @Override public void validate(Array expected, Array actual) throws SQLException {