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/04/07 17:16:54 UTC

[calcite-avatica] branch master updated: [CALCITE-4503] Order of fields in records should follow that of the SQL 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


The following commit(s) were added to refs/heads/master by this push:
     new 9e37120  [CALCITE-4503] Order of fields in records should follow that of the SQL types (Alessandro Solimando)
9e37120 is described below

commit 9e37120b1c6512354357f83dce0abb85176fc2c3
Author: Alessandro Solimando <18...@users.noreply.github.com>
AuthorDate: Sat Feb 6 19:30:55 2021 +0100

    [CALCITE-4503] Order of fields in records should follow that of the SQL types (Alessandro Solimando)
    
    1. RECORD and RECORD_PROJECTION are now handled in the same way (both require fields)
    so there is no point keeping both.
    2. Adapt LocalService#toResponse after dropping RECORD_PROJECTION to prevent broken clients,
    and add notes about the change of CursorFactory.
    3. Update history.md with details on the breaking change.
    4. Add javadoc for deduce method.
    
    Close apache/calcite-avatica#138
---
 .../main/java/org/apache/calcite/avatica/Meta.java |  33 ++-
 .../java/org/apache/calcite/avatica/MetaImpl.java  |  22 +-
 .../calcite/avatica/remote/LocalService.java       |  30 +--
 .../calcite/avatica/util/RecordIteratorCursor.java |   2 +
 .../calcite/avatica/CursorFactoryDeduceTest.java   | 222 +++++++++++++++++++++
 .../calcite/avatica/MetaImplCollectTest.java       | 113 +++++++++++
 site/_docs/history.md                              |  10 +
 7 files changed, 392 insertions(+), 40 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/avatica/Meta.java b/core/src/main/java/org/apache/calcite/avatica/Meta.java
index d69fb1c..a4f45c6 100644
--- a/core/src/main/java/org/apache/calcite/avatica/Meta.java
+++ b/core/src/main/java/org/apache/calcite/avatica/Meta.java
@@ -41,6 +41,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Properties;
+import java.util.stream.Collectors;
 
 /**
  * Command handler for getting various metadata. Should be implemented by each
@@ -607,9 +608,8 @@ public interface Meta {
 
     private CursorFactory(Style style, Class clazz, List<Field> fields,
         List<String> fieldNames) {
-      assert (fieldNames != null)
-          == (style == Style.RECORD_PROJECTION || style == Style.MAP);
-      assert (fields != null) == (style == Style.RECORD_PROJECTION);
+      assert (fieldNames != null) == (style == Style.RECORD || style == Style.MAP);
+      assert (fields != null) == (style == Style.RECORD);
       this.style = Objects.requireNonNull(style);
       this.clazz = clazz;
       this.fields = fields;
@@ -628,8 +628,6 @@ public interface Meta {
       case LIST:
         return LIST;
       case RECORD:
-        return record(clazz);
-      case RECORD_PROJECTION:
         return record(clazz, null, fieldNames);
       case MAP:
         return map(fieldNames);
@@ -647,8 +645,14 @@ public interface Meta {
     public static final CursorFactory LIST =
         new CursorFactory(Style.LIST, null, null, null);
 
+    /**
+     *
+     * @deprecated Use {@link #record(Class, List, List)}
+     */
+    @Deprecated // to be removed before 1.19.0
     public static CursorFactory record(Class resultClazz) {
-      return new CursorFactory(Style.RECORD, resultClazz, null, null);
+      List<Field> fields = Arrays.asList(resultClazz.getFields());
+      return new CursorFactory(Style.RECORD, resultClazz, fields, null);
     }
 
     public static CursorFactory record(Class resultClass, List<Field> fields,
@@ -663,14 +667,23 @@ public interface Meta {
           }
         }
       }
-      return new CursorFactory(Style.RECORD_PROJECTION, resultClass, fields,
-          fieldNames);
+      return new CursorFactory(Style.RECORD, resultClass, fields, fieldNames);
     }
 
     public static CursorFactory map(List<String> fieldNames) {
       return new CursorFactory(Style.MAP, null, null, fieldNames);
     }
 
+    /**
+     * Deduces the appropriate {@code CursorFactory} for accessing the underlying
+     * result set. For result sets composed by records, {@code resultClazz} must
+     * be not null, and each field name in {@code columns} must match one of its
+     * public fields.
+     * @param columns The columns metadata for the result set
+     * @param resultClazz The class representing the records, if any
+     * @return the appropriate {@code CursorFactory} for the underlying result set
+     * @throws RuntimeException if the field name validation fails
+     */
     public static CursorFactory deduce(List<ColumnMetaData> columns,
         Class resultClazz) {
       if (columns.size() == 1) {
@@ -685,7 +698,8 @@ public interface Meta {
       if (List.class.isAssignableFrom(resultClazz)) {
         return LIST;
       }
-      return record(resultClazz);
+      return record(resultClazz, null,
+          columns.stream().map(c -> c.columnName).collect(Collectors.toList()));
     }
 
     public Common.CursorFactory toProto() {
@@ -737,7 +751,6 @@ public interface Meta {
   enum Style {
     OBJECT,
     RECORD,
-    RECORD_PROJECTION,
     ARRAY,
     LIST,
     MAP;
diff --git a/core/src/main/java/org/apache/calcite/avatica/MetaImpl.java b/core/src/main/java/org/apache/calcite/avatica/MetaImpl.java
index f4f3e12..1b3b20e 100644
--- a/core/src/main/java/org/apache/calcite/avatica/MetaImpl.java
+++ b/core/src/main/java/org/apache/calcite/avatica/MetaImpl.java
@@ -89,14 +89,8 @@ public abstract class MetaImpl implements Meta {
           (Iterable<Object[]>) (Iterable) iterable;
       return new ArrayIteratorCursor(iterable1.iterator());
     case RECORD:
-      @SuppressWarnings("unchecked") final Class<Object> clazz =
-          cursorFactory.clazz;
-      return new RecordIteratorCursor<Object>(iterable.iterator(), clazz);
-    case RECORD_PROJECTION:
-      @SuppressWarnings("unchecked") final Class<Object> clazz2 =
-          cursorFactory.clazz;
-      return new RecordIteratorCursor<Object>(iterable.iterator(), clazz2,
-          cursorFactory.fields);
+      @SuppressWarnings("unchecked") final Class<Object> clazz = cursorFactory.clazz;
+      return new RecordIteratorCursor<>(iterable.iterator(), clazz, cursorFactory.fields);
     case LIST:
       @SuppressWarnings("unchecked") final Iterable<List<Object>> iterable2 =
           (Iterable<List<Object>>) (Iterable) iterable;
@@ -138,16 +132,8 @@ public abstract class MetaImpl implements Meta {
       }
       return list;
     case RECORD:
-    case RECORD_PROJECTION:
-      final Field[] fields;
-      switch (cursorFactory.style) {
-      case RECORD:
-        fields = cursorFactory.clazz.getFields();
-        break;
-      default:
-        fields = cursorFactory.fields.toArray(
-            new Field[cursorFactory.fields.size()]);
-      }
+      final Field[] fields = cursorFactory.fields.toArray(
+          new Field[cursorFactory.fields.size()]);
       for (Object o : iterable) {
         final Object[] objects = new Object[fields.length];
         for (int i = 0; i < fields.length; i++) {
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/LocalService.java b/core/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
index 929830b..df7ed47 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/LocalService.java
@@ -90,6 +90,8 @@ public class LocalService implements Service {
     }
 
     Meta.Signature signature = resultSet.signature;
+    // TODO Revise modification of CursorFactory see:
+    // https://issues.apache.org/jira/browse/CALCITE-4567
     Meta.CursorFactory cursorFactory = resultSet.signature.cursorFactory;
     Meta.Frame frame = null;
     int updateCount = -1;
@@ -97,20 +99,24 @@ public class LocalService implements Service {
 
     if (resultSet.firstFrame != null) {
       list = list(resultSet.firstFrame.rows);
-      switch (cursorFactory.style) {
-      case ARRAY:
+      if (list.isEmpty()) {
         cursorFactory = Meta.CursorFactory.LIST;
-        break;
-      case MAP:
-      case LIST:
-        break;
-      case RECORD:
-        cursorFactory = Meta.CursorFactory.LIST;
-        break;
-      default:
-        cursorFactory = Meta.CursorFactory.map(cursorFactory.fieldNames);
+      } else {
+        switch (cursorFactory.style) {
+        case ARRAY:
+          cursorFactory = Meta.CursorFactory.LIST;
+          break;
+        case MAP:
+        case LIST:
+          break;
+        case RECORD:
+          cursorFactory = Meta.CursorFactory.map(cursorFactory.fieldNames);
+          break;
+        default:
+          throw new IllegalStateException("Unknown cursor factory style: "
+              + cursorFactory.style);
+        }
       }
-
       final boolean done = resultSet.firstFrame.done;
 
       frame = new Meta.Frame(0, done, list);
diff --git a/core/src/main/java/org/apache/calcite/avatica/util/RecordIteratorCursor.java b/core/src/main/java/org/apache/calcite/avatica/util/RecordIteratorCursor.java
index 717247d..2646518 100644
--- a/core/src/main/java/org/apache/calcite/avatica/util/RecordIteratorCursor.java
+++ b/core/src/main/java/org/apache/calcite/avatica/util/RecordIteratorCursor.java
@@ -37,7 +37,9 @@ public class RecordIteratorCursor<E> extends IteratorCursor<E> {
    *
    * @param iterator Iterator
    * @param clazz Element type
+   * @deprecated Use {@link #RecordIteratorCursor(Iterator, Class, List)}
    */
+  @Deprecated // to be removed before 2.0
   public RecordIteratorCursor(Iterator<E> iterator, Class<E> clazz) {
     this(iterator, clazz, Arrays.asList(clazz.getFields()));
   }
diff --git a/core/src/test/java/org/apache/calcite/avatica/CursorFactoryDeduceTest.java b/core/src/test/java/org/apache/calcite/avatica/CursorFactoryDeduceTest.java
new file mode 100644
index 0000000..11f0464
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/avatica/CursorFactoryDeduceTest.java
@@ -0,0 +1,222 @@
+/*
+ * 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.util.Cursor;
+import org.apache.calcite.avatica.util.Unsafe;
+
+import org.junit.Test;
+
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@code Meta.CursorFactory} relative to deducing a cursor factory
+ * from the columns metadata.
+ */
+public class CursorFactoryDeduceTest {
+
+  static final ColumnMetaData.AvaticaType INT_TYPE =
+      ColumnMetaData.scalar(Types.INTEGER, "INT", ColumnMetaData.Rep.PRIMITIVE_INT);
+  static final ColumnMetaData.AvaticaType STRING_TYPE =
+      ColumnMetaData.scalar(Types.VARCHAR, "STRING", ColumnMetaData.Rep.STRING);
+  static final ColumnMetaData.AvaticaType DOUBLE_TYPE =
+      ColumnMetaData.scalar(Types.DOUBLE, "DOUBLE", ColumnMetaData.Rep.DOUBLE);
+
+  static final List<Object> ROWS = IntStream.range(1, 5)
+      .mapToObj(i -> (Object) new SimplePOJO(i, Integer.toString(i), (double) i))
+      .collect(Collectors.toList());
+
+  /**
+   * Simple POJO for testing cursors over Java objects.
+   */
+  protected static class SimplePOJO {
+    public int intField;
+    public String stringField;
+    public Double doubleField;
+
+    SimplePOJO(int intField, String stringField, Double doubleField) {
+      this.intField = intField;
+      this.stringField = stringField;
+      this.doubleField = doubleField;
+    }
+
+    @Override public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+
+      if (o == null) {
+        return false;
+      }
+
+      if (getClass() != o.getClass()) {
+        return false;
+      }
+
+      SimplePOJO pjo = (SimplePOJO) o;
+
+      return Objects.equals(stringField, pjo.stringField)
+          && Objects.equals(intField, pjo.intField)
+          && Objects.equals(doubleField, pjo.doubleField);
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(stringField, intField, doubleField);
+    }
+  }
+
+  /**
+   * Deducing the cursor from a single column having a Java object as type.
+   */
+  @Test public void deduceObjectCursorFactory() throws Exception {
+    ColumnMetaData.Rep rep = ColumnMetaData.Rep.of(SimplePOJO.class);
+    ColumnMetaData.AvaticaType pojoType =
+        ColumnMetaData.scalar(Types.OTHER, "OTHER", rep);
+
+    ColumnMetaData pojoMetadata =
+        MetaImpl.columnMetaData("POJO", 1, pojoType, true);
+
+    List<ColumnMetaData> columnMetaDataList = Collections.singletonList(pojoMetadata);
+
+    Meta.CursorFactory cursorFactory =
+        Meta.CursorFactory.deduce(columnMetaDataList, SimplePOJO.class);
+
+    try (Cursor cursor = MetaImpl.createCursor(cursorFactory, ROWS)) {
+      List<Cursor.Accessor> accessors =
+          cursor.createAccessors(columnMetaDataList, Unsafe.localCalendar(), null);
+
+      assertEquals(1, accessors.size());
+      Cursor.Accessor accessor = accessors.get(0);
+
+      for (Object row : ROWS) {
+        assertTrue(cursor.next());
+        assertEquals(row, accessor.getObject());
+      }
+
+      assertFalse(cursor.next());
+    }
+  }
+
+  /**
+   * Deducing the cursor when columns are the fields of a Java object.
+   */
+  @Test public void deduceRecordCursorFactory() throws Exception {
+    List<ColumnMetaData> columnsMetaDataList = Arrays.asList(
+        MetaImpl.columnMetaData("intField", 1, INT_TYPE, true),
+        MetaImpl.columnMetaData("stringField", 2, STRING_TYPE, true),
+        MetaImpl.columnMetaData("doubleField", 3, DOUBLE_TYPE, true));
+
+    Meta.CursorFactory cursorFactory =
+        Meta.CursorFactory.deduce(columnsMetaDataList, SimplePOJO.class);
+
+    try (Cursor cursor = MetaImpl.createCursor(cursorFactory, ROWS)) {
+      List<Cursor.Accessor> accessors =
+          cursor.createAccessors(columnsMetaDataList, Unsafe.localCalendar(), null);
+
+      assertEquals(columnsMetaDataList.size(), accessors.size());
+      Cursor.Accessor intAccessor = accessors.get(0);
+      Cursor.Accessor strAccessor = accessors.get(1);
+      Cursor.Accessor doubleAccessor = accessors.get(2);
+
+      for (Object row : ROWS) {
+        assertTrue(cursor.next());
+        SimplePOJO pjo = (SimplePOJO) row;
+        assertEquals(pjo.intField, intAccessor.getObject());
+        assertEquals(pjo.stringField, strAccessor.getObject());
+        assertEquals(pjo.doubleField, doubleAccessor.getObject());
+      }
+
+      assertFalse(cursor.next());
+    }
+  }
+
+  /**
+   * Deducing the cursor when columns are the fields of a Java object,
+   * different columns ordering.
+   */
+  @Test public void deduceRecordCursorFactoryDifferentFieldsOrdering() throws Exception {
+    List<ColumnMetaData> columnsMetaDataList = Arrays.asList(
+        MetaImpl.columnMetaData("stringField", 2, STRING_TYPE, true),
+        MetaImpl.columnMetaData("doubleField", 3, DOUBLE_TYPE, true),
+        MetaImpl.columnMetaData("intField", 1, INT_TYPE, true));
+
+    Meta.CursorFactory cursorFactory =
+        Meta.CursorFactory.deduce(columnsMetaDataList, SimplePOJO.class);
+
+    try (Cursor cursor = MetaImpl.createCursor(cursorFactory, ROWS)) {
+      List<Cursor.Accessor> accessors =
+          cursor.createAccessors(columnsMetaDataList, Unsafe.localCalendar(), null);
+
+      assertEquals(columnsMetaDataList.size(), accessors.size());
+      Cursor.Accessor strAccessor = accessors.get(0);
+      Cursor.Accessor doubleAccessor = accessors.get(1);
+      Cursor.Accessor intAccessor = accessors.get(2);
+
+      for (Object row : ROWS) {
+        assertTrue(cursor.next());
+        SimplePOJO pjo = (SimplePOJO) row;
+        assertEquals(pjo.intField, intAccessor.getObject());
+        assertEquals(pjo.stringField, strAccessor.getObject());
+        assertEquals(pjo.doubleField, doubleAccessor.getObject());
+      }
+
+      assertFalse(cursor.next());
+    }
+  }
+
+  /**
+   * Deducing the cursor when columns are (a subset of) the fields of a Java object.
+   */
+  @Test public void deduceRecordCursorFactoryProjectedFields() throws Exception {
+    List<ColumnMetaData> columnsMetaDataList = Arrays.asList(
+        MetaImpl.columnMetaData("stringField", 1, STRING_TYPE, true),
+        MetaImpl.columnMetaData("doubleField", 2, DOUBLE_TYPE, true));
+
+    Meta.CursorFactory cursorFactory =
+        Meta.CursorFactory.deduce(columnsMetaDataList, SimplePOJO.class);
+
+    try (Cursor cursor = MetaImpl.createCursor(cursorFactory, ROWS)) {
+      List<Cursor.Accessor> accessors =
+          cursor.createAccessors(columnsMetaDataList, Unsafe.localCalendar(), null);
+
+      assertEquals(columnsMetaDataList.size(), accessors.size());
+      Cursor.Accessor strAccessor = accessors.get(0);
+      Cursor.Accessor doubleAccessor = accessors.get(1);
+
+      for (Object row : ROWS) {
+        assertTrue(cursor.next());
+        SimplePOJO pjo = (SimplePOJO) row;
+        assertEquals(pjo.stringField, strAccessor.getObject());
+        assertEquals(pjo.doubleField, doubleAccessor.getObject());
+      }
+
+      assertFalse(cursor.next());
+    }
+  }
+}
+
+// End CursorFactoryDeduceTest.java
diff --git a/core/src/test/java/org/apache/calcite/avatica/MetaImplCollectTest.java b/core/src/test/java/org/apache/calcite/avatica/MetaImplCollectTest.java
new file mode 100644
index 0000000..c423f29
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/avatica/MetaImplCollectTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.junit.Test;
+
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.calcite.avatica.CursorFactoryDeduceTest.DOUBLE_TYPE;
+import static org.apache.calcite.avatica.CursorFactoryDeduceTest.INT_TYPE;
+import static org.apache.calcite.avatica.CursorFactoryDeduceTest.ROWS;
+import static org.apache.calcite.avatica.CursorFactoryDeduceTest.STRING_TYPE;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@code MetaImpl} relative to the {@code collect} method.
+ */
+public class MetaImplCollectTest {
+
+  /**
+   * Collecting from records where columns are the fields of a Java object.
+   */
+  @Test
+  public void collectRecord() {
+    List<ColumnMetaData> columnsMetaDataList = Arrays.asList(
+        MetaImpl.columnMetaData("stringField", 2, STRING_TYPE, true),
+        MetaImpl.columnMetaData("doubleField", 3, DOUBLE_TYPE, true),
+        MetaImpl.columnMetaData("intField", 1, INT_TYPE, true));
+
+    Meta.CursorFactory cursorFactory =
+        Meta.CursorFactory.deduce(columnsMetaDataList, CursorFactoryDeduceTest.SimplePOJO.class);
+
+    List<List<Object>> rows = new ArrayList<>();
+    MetaImpl.collect(cursorFactory, ROWS, rows);
+
+    for (int i = 0; i < ROWS.size(); i++) {
+      CursorFactoryDeduceTest.SimplePOJO inputRow =
+          (CursorFactoryDeduceTest.SimplePOJO) ROWS.get(i);
+      List<Object> collectedRow = rows.get(i);
+      assertEquals(inputRow.stringField, collectedRow.get(0));
+      assertEquals(inputRow.doubleField, collectedRow.get(1));
+      assertEquals(inputRow.intField, collectedRow.get(2));
+    }
+  }
+
+  /**
+   * Collecting from records where columns are (a subset of) the fields of a Java object.
+   */
+  @Test public void collectProjectedRecord() {
+    List<ColumnMetaData> columnsMetaDataList = Arrays.asList(
+        MetaImpl.columnMetaData("stringField", 2, STRING_TYPE, true),
+        MetaImpl.columnMetaData("doubleField", 3, DOUBLE_TYPE, true));
+
+    Meta.CursorFactory cursorFactory =
+        Meta.CursorFactory.deduce(columnsMetaDataList, CursorFactoryDeduceTest.SimplePOJO.class);
+
+    List<List<Object>> rows = new ArrayList<>();
+    MetaImpl.collect(cursorFactory, ROWS, rows);
+
+    for (int i = 0; i < ROWS.size(); i++) {
+      CursorFactoryDeduceTest.SimplePOJO inputRow =
+          (CursorFactoryDeduceTest.SimplePOJO) ROWS.get(i);
+      List<Object> collectedRow = rows.get(i);
+      assertEquals(inputRow.stringField, collectedRow.get(0));
+      assertEquals(inputRow.doubleField, collectedRow.get(1));
+    }
+  }
+
+  /**
+   * Collect from a single column having a Java object as type.
+   */
+  @Test public void collectObject() {
+    ColumnMetaData.Rep rep = ColumnMetaData.Rep.of(CursorFactoryDeduceTest.SimplePOJO.class);
+    ColumnMetaData.AvaticaType pojoType =
+        ColumnMetaData.scalar(Types.OTHER, "OTHER", rep);
+
+    ColumnMetaData pojoMetadata =
+        MetaImpl.columnMetaData("POJO", 1, pojoType, true);
+
+    List<ColumnMetaData> columnMetaDataList = Collections.singletonList(pojoMetadata);
+
+    Meta.CursorFactory cursorFactory =
+        Meta.CursorFactory.deduce(columnMetaDataList, CursorFactoryDeduceTest.SimplePOJO.class);
+
+    List<List<Object>> rows = new ArrayList<>();
+    MetaImpl.collect(cursorFactory, ROWS, rows);
+
+    for (int i = 0; i < ROWS.size(); i++) {
+      assertEquals(ROWS.get(i), rows.get(i).get(0));
+    }
+  }
+}
+
+// End MetaImplCollectTest.java
diff --git a/site/_docs/history.md b/site/_docs/history.md
index eb3311c..397315b 100644
--- a/site/_docs/history.md
+++ b/site/_docs/history.md
@@ -28,6 +28,16 @@ For a full list of releases, see
 Downloads are available on the
 [downloads page]({{ site.baseurl }}/downloads/avatica.html).
 
+## <a href="https://github.com/apache/calcite-avatica/releases/tag/rel/avatica-1.18.0">1.18.0</a> / 2021-04-XX
+{: #v1-18-0}
+
+### Breaking changes
+* [<a href="https://issues.apache.org/jira/browse/CALCITE-4503">CALCITE-4503</a>] Order of fields in records should follow that of the SQL types
+
+`Meta#deduce(List<ColumnMetaData> columns, Class resultClazz)` now only derives the order of the fields from the list of provided column metadata `columns` when generating a record from the given Java class `resultClazz`, instead of relying on the field order provided by `Object#getFields()`, which is a JVM-dependent feature.
+
+Before, the field names where not checked against the field names of the provided class. Now, if `resultClazz` is not null, the provided field names are expected to match existing fields in that class. If a column metadata has name `column`, and no public field in `resultClazz` with that name exists, the following exception is thrown:
+`java.lang.RuntimeException: java.lang.NoSuchFieldException: C`.
 
 ## <a href="https://github.com/apache/calcite-avatica/releases/tag/rel/avatica-1.17.0">1.17.0</a> / 2020-06-22
 {: #v1-17-0}