You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/08/17 15:14:22 UTC

[GitHub] [arrow] igor-suhorukov opened a new pull request, #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

igor-suhorukov opened a new pull request, #13906:
URL: https://github.com/apache/arrow/pull/13906

   Typical real life Arrow datasets contain List type vectors of primitive type. This PR introduce ListBinder mapping of primitive types lists to java.sql.Types.ARRAY


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] igor-suhorukov commented on a diff in pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
igor-suhorukov commented on code in PR #13906:
URL: https://github.com/apache/arrow/pull/13906#discussion_r949426997


##########
java/adapter/jdbc/src/test/java/org/apache/arrow/adapter/jdbc/JdbcParameterBinderTest.java:
##########
@@ -483,4 +571,105 @@ <T, V extends FieldVector> void testSimpleType(ArrowType arrowType, int jdbcType
       assertThat(binder.next()).isFalse();
     }
   }
+
+  <T, V extends FieldVector> void testListType(ArrowType arrowType, TriConsumer<V, Integer, T> setValue,
+                          BiConsumer<V, Integer> setNull, List<T> values) throws SQLException {
+    int jdbcType = Types.ARRAY;
+    Schema schema = new Schema(Collections.singletonList(new Field("field", FieldType.nullable(
+            new ArrowType.List()), Collections.singletonList(
+            new Field("element", FieldType.notNullable(arrowType), null)
+    ))));
+    try (final MockPreparedStatement statement = new MockPreparedStatement();
+         final VectorSchemaRoot root = VectorSchemaRoot.create(schema, allocator)) {
+      final JdbcParameterBinder binder =
+          JdbcParameterBinder.builder(statement, root).bindAll().build();
+      assertThat(binder.next()).isFalse();
+
+      @SuppressWarnings("unchecked")
+      final V vector = (V) root.getVector(0);
+      final ColumnBinder columnBinder = ColumnBinder.forVector(vector);
+      assertThat(columnBinder.getJdbcType()).isEqualTo(jdbcType);
+
+      setValue.accept(vector, 0, values.get(0));
+      setValue.accept(vector, 1, values.get(1));
+      setNull.accept(vector, 2);
+      root.setRowCount(3);
+
+      assertThat(binder.next()).isTrue();
+      assertThat(statement.getParamValue(1)).isEqualTo(values.get(0));
+      assertThat(binder.next()).isTrue();
+      assertThat(statement.getParamValue(1)).isEqualTo(values.get(1));
+      assertThat(binder.next()).isTrue();
+      assertThat(statement.getParamValue(1)).isNull();
+      assertThat(statement.getParamType(1)).isEqualTo(jdbcType);
+      assertThat(binder.next()).isFalse();
+
+      binder.reset();
+
+      setNull.accept(vector, 0);
+      setValue.accept(vector, 1, values.get(2));
+      setValue.accept(vector, 2, values.get(0));
+      setValue.accept(vector, 3, values.get(2));

Review Comment:
   Thanks you @lidavidm . Test fixed by last commit



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] ursabot commented on pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
ursabot commented on PR #13906:
URL: https://github.com/apache/arrow/pull/13906#issuecomment-1220738763

   ['Python', 'R'] benchmarks have high level of regressions.
   [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/1dfcd406e29e4d4f857770b91d5a24f4...6cc48394e762478f80961c4996813cea/)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] igor-suhorukov commented on a diff in pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
igor-suhorukov commented on code in PR #13906:
URL: https://github.com/apache/arrow/pull/13906#discussion_r949146357


##########
java/adapter/jdbc/src/test/java/org/apache/arrow/adapter/jdbc/JdbcParameterBinderTest.java:
##########
@@ -385,6 +388,106 @@ void decimal256() throws SQLException {
         Arrays.asList(new BigDecimal("120.429"), new BigDecimal("-10590.123"), new BigDecimal("0.000")));
   }
 
+  @Test
+  void listOfDouble() throws SQLException {
+    TriConsumer<ListVector, Integer, Double[]> setValue = (listVector, index, values) -> {
+      org.apache.arrow.vector.complex.impl.UnionListWriter writer = listVector.getWriter();
+      writer.setPosition(index);
+      writer.startList();
+      Arrays.stream(values).forEach(doubleValue -> writer.float8().writeFloat8(doubleValue));
+      writer.endList();
+      listVector.setLastSet(index);
+    };
+    List<Double[]> values = Arrays.asList(new Double[]{0.0, Math.PI}, new Double[]{1.1, -352346.2, 2355.6},
+                                          new Double[]{-1024.3});
+    testListType(new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE), setValue, ListVector::setNull, values);
+  }
+
+  @Test
+  void listOfInt64() throws SQLException {
+    TriConsumer<ListVector, Integer, Long[]> setValue = (listVector, index, values) -> {
+      org.apache.arrow.vector.complex.impl.UnionListWriter writer = listVector.getWriter();
+      writer.setPosition(index);
+      writer.startList();
+      Arrays.stream(values).forEach(longValue -> writer.bigInt().writeBigInt(longValue));
+      writer.endList();
+      listVector.setLastSet(index);
+    };
+    List<Long[]> values = Arrays.asList(new Long[]{1L, 2L, 3L}, new Long[]{4L, 5L},
+            new Long[]{512L, 1024L, 2048L, 4096L});
+    testListType((ArrowType) new ArrowType.Int(64, true), setValue, ListVector::setNull, values);
+  }
+
+  @Test
+  void listOfInt32() throws SQLException {
+    TriConsumer<ListVector, Integer, Integer[]> setValue = (listVector, index, values) -> {
+      org.apache.arrow.vector.complex.impl.UnionListWriter writer = listVector.getWriter();
+      writer.setPosition(index);
+      writer.startList();
+      Arrays.stream(values).forEach(integerValue -> writer.integer().writeInt(integerValue));
+      writer.endList();
+      listVector.setLastSet(index);
+    };
+    List<Integer[]> values = Arrays.asList(new Integer[]{1, 2, 3}, new Integer[]{4, 5},
+            new Integer[]{512, 1024, 2048, 4096});
+    testListType((ArrowType) new ArrowType.Int(32, true), setValue, ListVector::setNull, values);
+  }
+
+  @Test
+  void listOfInt32WithEmptyList() throws SQLException {

Review Comment:
   Sure, thanks for advice



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] igor-suhorukov commented on a diff in pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
igor-suhorukov commented on code in PR #13906:
URL: https://github.com/apache/arrow/pull/13906#discussion_r949156691


##########
java/adapter/jdbc/src/test/java/org/apache/arrow/adapter/jdbc/JdbcParameterBinderTest.java:
##########
@@ -385,6 +388,106 @@ void decimal256() throws SQLException {
         Arrays.asList(new BigDecimal("120.429"), new BigDecimal("-10590.123"), new BigDecimal("0.000")));
   }
 
+  @Test
+  void listOfDouble() throws SQLException {
+    TriConsumer<ListVector, Integer, Double[]> setValue = (listVector, index, values) -> {
+      org.apache.arrow.vector.complex.impl.UnionListWriter writer = listVector.getWriter();
+      writer.setPosition(index);
+      writer.startList();
+      Arrays.stream(values).forEach(doubleValue -> writer.float8().writeFloat8(doubleValue));
+      writer.endList();
+      listVector.setLastSet(index);
+    };
+    List<Double[]> values = Arrays.asList(new Double[]{0.0, Math.PI}, new Double[]{1.1, -352346.2, 2355.6},
+                                          new Double[]{-1024.3});
+    testListType(new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE), setValue, ListVector::setNull, values);
+  }
+
+  @Test
+  void listOfInt64() throws SQLException {
+    TriConsumer<ListVector, Integer, Long[]> setValue = (listVector, index, values) -> {
+      org.apache.arrow.vector.complex.impl.UnionListWriter writer = listVector.getWriter();
+      writer.setPosition(index);
+      writer.startList();
+      Arrays.stream(values).forEach(longValue -> writer.bigInt().writeBigInt(longValue));
+      writer.endList();
+      listVector.setLastSet(index);
+    };
+    List<Long[]> values = Arrays.asList(new Long[]{1L, 2L, 3L}, new Long[]{4L, 5L},
+            new Long[]{512L, 1024L, 2048L, 4096L});
+    testListType((ArrowType) new ArrowType.Int(64, true), setValue, ListVector::setNull, values);
+  }
+
+  @Test
+  void listOfInt32() throws SQLException {
+    TriConsumer<ListVector, Integer, Integer[]> setValue = (listVector, index, values) -> {
+      org.apache.arrow.vector.complex.impl.UnionListWriter writer = listVector.getWriter();
+      writer.setPosition(index);
+      writer.startList();
+      Arrays.stream(values).forEach(integerValue -> writer.integer().writeInt(integerValue));
+      writer.endList();
+      listVector.setLastSet(index);
+    };
+    List<Integer[]> values = Arrays.asList(new Integer[]{1, 2, 3}, new Integer[]{4, 5},
+            new Integer[]{512, 1024, 2048, 4096});
+    testListType((ArrowType) new ArrowType.Int(32, true), setValue, ListVector::setNull, values);
+  }
+
+  @Test
+  void listOfInt32WithEmptyList() throws SQLException {

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on a diff in pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13906:
URL: https://github.com/apache/arrow/pull/13906#discussion_r949126301


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/ListBinder.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.arrow.adapter.jdbc.binder;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.impl.UnionListReader;
+import org.apache.arrow.vector.util.Text;
+
+/**
+ * A column binder for list of primitive values.
+ */
+public class ListBinder extends BaseColumnBinder<ListVector> {
+
+  private final UnionListReader listReader;
+  private final Class<?> arrayElementClass;
+  private final boolean isTextColumn;
+
+  public ListBinder(ListVector vector) {
+    this(vector, java.sql.Types.ARRAY);
+  }
+
+  /**
+   * Init ListBinder and determine type of data vector.
+   *
+   * @param vector corresponding data vector from arrow buffer for binding
+   * @param jdbcType parameter jdbc type
+   */
+  public ListBinder(ListVector vector, int jdbcType) {
+    super(vector, jdbcType);
+    listReader = vector.getReader();
+    Class<? extends FieldVector> dataVectorClass = vector.getDataVector().getClass();
+    try {
+      arrayElementClass = dataVectorClass.getMethod("getObject", Integer.TYPE).getReturnType();
+    } catch (NoSuchMethodException e) {
+      final String message = String.format("Issue to determine type for getObject method of data vector class %s ",
+              dataVectorClass.getName());
+      throw new RuntimeException(message);
+    }
+    isTextColumn = arrayElementClass.isAssignableFrom(Text.class);
+  }
+
+  @Override
+  public void bind(java.sql.PreparedStatement statement, int parameterIndex, int rowIndex)throws java.sql.SQLException {
+    listReader.setPosition(rowIndex);
+    ArrayList<?> sourceArray = (ArrayList<?>) listReader.readObject();
+    Object array;
+    if (!isTextColumn) {
+      array = Array.newInstance(arrayElementClass, sourceArray.size());
+      Arrays.setAll((Object[]) array, sourceArray::get);
+    } else {
+      array = new String[sourceArray.size()];
+      Arrays.setAll((Object[]) array, idx -> sourceArray.get(idx) != null ? sourceArray.get(idx).toString() : null);

Review Comment:
   nit: instead of calling `get` twice, use `sourceArray.isNull(idx)`



##########
java/adapter/jdbc/src/test/java/org/apache/arrow/adapter/jdbc/JdbcParameterBinderTest.java:
##########
@@ -385,6 +388,106 @@ void decimal256() throws SQLException {
         Arrays.asList(new BigDecimal("120.429"), new BigDecimal("-10590.123"), new BigDecimal("0.000")));
   }
 
+  @Test
+  void listOfDouble() throws SQLException {
+    TriConsumer<ListVector, Integer, Double[]> setValue = (listVector, index, values) -> {
+      org.apache.arrow.vector.complex.impl.UnionListWriter writer = listVector.getWriter();
+      writer.setPosition(index);
+      writer.startList();
+      Arrays.stream(values).forEach(doubleValue -> writer.float8().writeFloat8(doubleValue));
+      writer.endList();
+      listVector.setLastSet(index);
+    };
+    List<Double[]> values = Arrays.asList(new Double[]{0.0, Math.PI}, new Double[]{1.1, -352346.2, 2355.6},
+                                          new Double[]{-1024.3});
+    testListType(new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE), setValue, ListVector::setNull, values);
+  }
+
+  @Test
+  void listOfInt64() throws SQLException {
+    TriConsumer<ListVector, Integer, Long[]> setValue = (listVector, index, values) -> {
+      org.apache.arrow.vector.complex.impl.UnionListWriter writer = listVector.getWriter();
+      writer.setPosition(index);
+      writer.startList();
+      Arrays.stream(values).forEach(longValue -> writer.bigInt().writeBigInt(longValue));
+      writer.endList();
+      listVector.setLastSet(index);
+    };
+    List<Long[]> values = Arrays.asList(new Long[]{1L, 2L, 3L}, new Long[]{4L, 5L},
+            new Long[]{512L, 1024L, 2048L, 4096L});
+    testListType((ArrowType) new ArrowType.Int(64, true), setValue, ListVector::setNull, values);
+  }
+
+  @Test
+  void listOfInt32() throws SQLException {
+    TriConsumer<ListVector, Integer, Integer[]> setValue = (listVector, index, values) -> {
+      org.apache.arrow.vector.complex.impl.UnionListWriter writer = listVector.getWriter();
+      writer.setPosition(index);
+      writer.startList();
+      Arrays.stream(values).forEach(integerValue -> writer.integer().writeInt(integerValue));
+      writer.endList();
+      listVector.setLastSet(index);
+    };
+    List<Integer[]> values = Arrays.asList(new Integer[]{1, 2, 3}, new Integer[]{4, 5},
+            new Integer[]{512, 1024, 2048, 4096});
+    testListType((ArrowType) new ArrowType.Int(32, true), setValue, ListVector::setNull, values);
+  }
+
+  @Test
+  void listOfInt32WithEmptyList() throws SQLException {

Review Comment:
   nit: why not just add an empty list value to each of the test cases, instead of having a separate test only for one case?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13906:
URL: https://github.com/apache/arrow/pull/13906#issuecomment-1218243140

   https://issues.apache.org/jira/browse/ARROW-17430


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] igor-suhorukov commented on a diff in pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
igor-suhorukov commented on code in PR #13906:
URL: https://github.com/apache/arrow/pull/13906#discussion_r948465760


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/ListBinder.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.arrow.adapter.jdbc.binder;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.impl.UnionListReader;
+
+/**
+ * A column binder for list of primitive values.
+ */
+public class ListBinder extends BaseColumnBinder<ListVector> {
+
+  private UnionListReader listReader;
+
+  public ListBinder(ListVector vector) {
+    this(vector, java.sql.Types.ARRAY);
+  }
+
+  public ListBinder(ListVector vector, int jdbcType) {
+    super(vector, jdbcType);
+    listReader = vector.getReader();
+  }
+
+  @Override
+  public void bind(java.sql.PreparedStatement statement, int parameterIndex, int rowIndex)throws java.sql.SQLException {
+    listReader.setPosition(rowIndex);
+    ArrayList<?> sourceArray = (ArrayList<?>) listReader.readObject();
+    Class<?> arrayElementClass = sourceArray.get(0).getClass();

Review Comment:
   well spotted! I change implementation and test to process empty list case and Text values



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on a diff in pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13906:
URL: https://github.com/apache/arrow/pull/13906#discussion_r949147149


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/ListBinder.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.arrow.adapter.jdbc.binder;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.impl.UnionListReader;
+import org.apache.arrow.vector.util.Text;
+
+/**
+ * A column binder for list of primitive values.
+ */
+public class ListBinder extends BaseColumnBinder<ListVector> {
+
+  private final UnionListReader listReader;
+  private final Class<?> arrayElementClass;
+  private final boolean isTextColumn;
+
+  public ListBinder(ListVector vector) {
+    this(vector, java.sql.Types.ARRAY);
+  }
+
+  /**
+   * Init ListBinder and determine type of data vector.
+   *
+   * @param vector corresponding data vector from arrow buffer for binding
+   * @param jdbcType parameter jdbc type
+   */
+  public ListBinder(ListVector vector, int jdbcType) {
+    super(vector, jdbcType);
+    listReader = vector.getReader();
+    Class<? extends FieldVector> dataVectorClass = vector.getDataVector().getClass();
+    try {
+      arrayElementClass = dataVectorClass.getMethod("getObject", Integer.TYPE).getReturnType();
+    } catch (NoSuchMethodException e) {
+      final String message = String.format("Issue to determine type for getObject method of data vector class %s ",
+              dataVectorClass.getName());
+      throw new RuntimeException(message);
+    }
+    isTextColumn = arrayElementClass.isAssignableFrom(Text.class);
+  }
+
+  @Override
+  public void bind(java.sql.PreparedStatement statement, int parameterIndex, int rowIndex)throws java.sql.SQLException {
+    listReader.setPosition(rowIndex);
+    ArrayList<?> sourceArray = (ArrayList<?>) listReader.readObject();
+    Object array;
+    if (!isTextColumn) {
+      array = Array.newInstance(arrayElementClass, sourceArray.size());
+      Arrays.setAll((Object[]) array, sourceArray::get);
+    } else {
+      array = new String[sourceArray.size()];
+      Arrays.setAll((Object[]) array, idx -> sourceArray.get(idx) != null ? sourceArray.get(idx).toString() : null);

Review Comment:
   Oh whoops, I thought this was a ValueVector, not an ArrayList for some reason. No problems here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] ursabot commented on pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
ursabot commented on PR #13906:
URL: https://github.com/apache/arrow/pull/13906#issuecomment-1220738569

   Benchmark runs are scheduled for baseline = 1a34a078b29c3f119c93de10631d26fc0d52739a and contender = b11bc5054e9499bf022003aeb0352f12ddfc61fe. b11bc5054e9499bf022003aeb0352f12ddfc61fe is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/4ecfbaf2dac04549b01d11d91f471e4d...ce6b86a6120342369a5761e84d8bc7b1/)
   [Failed :arrow_down:0.27% :arrow_up:0.03%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/824efaba06254ad5b33f7afda172f236...7de3d650906e4d7283a0eb674849b5c9/)
   [Failed :arrow_down:0.82% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/1dfcd406e29e4d4f857770b91d5a24f4...6cc48394e762478f80961c4996813cea/)
   [Finished :arrow_down:0.46% :arrow_up:0.14%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/0b7dd51da38c4e41b70f676a8b6ab673...678683b8fb4841f1abf66999695f4e84/)
   Buildkite builds:
   [Finished] [`b11bc505` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/1337)
   [Failed] [`b11bc505` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/1353)
   [Failed] [`b11bc505` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/1335)
   [Finished] [`b11bc505` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/1352)
   [Finished] [`1a34a078` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/1336)
   [Finished] [`1a34a078` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/1352)
   [Failed] [`1a34a078` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/1334)
   [Finished] [`1a34a078` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/1351)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on a diff in pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13906:
URL: https://github.com/apache/arrow/pull/13906#discussion_r948142802


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/ListBinder.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.arrow.adapter.jdbc.binder;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.impl.UnionListReader;
+
+/**
+ * A column binder for list of primitive values.
+ */
+public class ListBinder extends BaseColumnBinder<ListVector> {
+
+  private UnionListReader listReader;
+
+  public ListBinder(ListVector vector) {
+    this(vector, java.sql.Types.ARRAY);
+  }
+
+  public ListBinder(ListVector vector, int jdbcType) {
+    super(vector, jdbcType);
+    listReader = vector.getReader();
+  }
+
+  @Override
+  public void bind(java.sql.PreparedStatement statement, int parameterIndex, int rowIndex)throws java.sql.SQLException {
+    listReader.setPosition(rowIndex);
+    ArrayList<?> sourceArray = (ArrayList<?>) listReader.readObject();
+    Class<?> arrayElementClass = sourceArray.get(0).getClass();

Review Comment:
   This won't work if the list element is empty, right?



##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/ListBinder.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.arrow.adapter.jdbc.binder;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.impl.UnionListReader;
+
+/**
+ * A column binder for list of primitive values.
+ */
+public class ListBinder extends BaseColumnBinder<ListVector> {
+
+  private UnionListReader listReader;
+
+  public ListBinder(ListVector vector) {
+    this(vector, java.sql.Types.ARRAY);
+  }
+
+  public ListBinder(ListVector vector, int jdbcType) {
+    super(vector, jdbcType);
+    listReader = vector.getReader();
+  }
+
+  @Override
+  public void bind(java.sql.PreparedStatement statement, int parameterIndex, int rowIndex)throws java.sql.SQLException {
+    listReader.setPosition(rowIndex);
+    ArrayList<?> sourceArray = (ArrayList<?>) listReader.readObject();
+    Class<?> arrayElementClass = sourceArray.get(0).getClass();

Review Comment:
   I think we'd also want to see that covered in the tests



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm merged pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
lidavidm merged PR #13906:
URL: https://github.com/apache/arrow/pull/13906


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] lidavidm commented on a diff in pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13906:
URL: https://github.com/apache/arrow/pull/13906#discussion_r949269533


##########
java/adapter/jdbc/src/test/java/org/apache/arrow/adapter/jdbc/JdbcParameterBinderTest.java:
##########
@@ -483,4 +571,105 @@ <T, V extends FieldVector> void testSimpleType(ArrowType arrowType, int jdbcType
       assertThat(binder.next()).isFalse();
     }
   }
+
+  <T, V extends FieldVector> void testListType(ArrowType arrowType, TriConsumer<V, Integer, T> setValue,
+                          BiConsumer<V, Integer> setNull, List<T> values) throws SQLException {
+    int jdbcType = Types.ARRAY;
+    Schema schema = new Schema(Collections.singletonList(new Field("field", FieldType.nullable(
+            new ArrowType.List()), Collections.singletonList(
+            new Field("element", FieldType.notNullable(arrowType), null)
+    ))));
+    try (final MockPreparedStatement statement = new MockPreparedStatement();
+         final VectorSchemaRoot root = VectorSchemaRoot.create(schema, allocator)) {
+      final JdbcParameterBinder binder =
+          JdbcParameterBinder.builder(statement, root).bindAll().build();
+      assertThat(binder.next()).isFalse();
+
+      @SuppressWarnings("unchecked")
+      final V vector = (V) root.getVector(0);
+      final ColumnBinder columnBinder = ColumnBinder.forVector(vector);
+      assertThat(columnBinder.getJdbcType()).isEqualTo(jdbcType);
+
+      setValue.accept(vector, 0, values.get(0));
+      setValue.accept(vector, 1, values.get(1));
+      setNull.accept(vector, 2);
+      root.setRowCount(3);
+
+      assertThat(binder.next()).isTrue();
+      assertThat(statement.getParamValue(1)).isEqualTo(values.get(0));
+      assertThat(binder.next()).isTrue();
+      assertThat(statement.getParamValue(1)).isEqualTo(values.get(1));
+      assertThat(binder.next()).isTrue();
+      assertThat(statement.getParamValue(1)).isNull();
+      assertThat(statement.getParamType(1)).isEqualTo(jdbcType);
+      assertThat(binder.next()).isFalse();
+
+      binder.reset();
+
+      setNull.accept(vector, 0);
+      setValue.accept(vector, 1, values.get(2));
+      setValue.accept(vector, 2, values.get(0));
+      setValue.accept(vector, 3, values.get(2));

Review Comment:
   The empty array value isn't being used, since the testers here only access indices 0-2. Maybe change one of these indices to `3`?  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13906:
URL: https://github.com/apache/arrow/pull/13906#issuecomment-1218243168

   :warning: Ticket **has not been started in JIRA**, please click 'Start Progress'.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] igor-suhorukov commented on a diff in pull request #13906: ARROW-17430: [Java] ListBinder to bind Arrow List type to DB column

Posted by GitBox <gi...@apache.org>.
igor-suhorukov commented on code in PR #13906:
URL: https://github.com/apache/arrow/pull/13906#discussion_r949145695


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/ListBinder.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.arrow.adapter.jdbc.binder;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.impl.UnionListReader;
+import org.apache.arrow.vector.util.Text;
+
+/**
+ * A column binder for list of primitive values.
+ */
+public class ListBinder extends BaseColumnBinder<ListVector> {
+
+  private final UnionListReader listReader;
+  private final Class<?> arrayElementClass;
+  private final boolean isTextColumn;
+
+  public ListBinder(ListVector vector) {
+    this(vector, java.sql.Types.ARRAY);
+  }
+
+  /**
+   * Init ListBinder and determine type of data vector.
+   *
+   * @param vector corresponding data vector from arrow buffer for binding
+   * @param jdbcType parameter jdbc type
+   */
+  public ListBinder(ListVector vector, int jdbcType) {
+    super(vector, jdbcType);
+    listReader = vector.getReader();
+    Class<? extends FieldVector> dataVectorClass = vector.getDataVector().getClass();
+    try {
+      arrayElementClass = dataVectorClass.getMethod("getObject", Integer.TYPE).getReturnType();
+    } catch (NoSuchMethodException e) {
+      final String message = String.format("Issue to determine type for getObject method of data vector class %s ",
+              dataVectorClass.getName());
+      throw new RuntimeException(message);
+    }
+    isTextColumn = arrayElementClass.isAssignableFrom(Text.class);
+  }
+
+  @Override
+  public void bind(java.sql.PreparedStatement statement, int parameterIndex, int rowIndex)throws java.sql.SQLException {
+    listReader.setPosition(rowIndex);
+    ArrayList<?> sourceArray = (ArrayList<?>) listReader.readObject();
+    Object array;
+    if (!isTextColumn) {
+      array = Array.newInstance(arrayElementClass, sourceArray.size());
+      Arrays.setAll((Object[]) array, sourceArray::get);
+    } else {
+      array = new String[sourceArray.size()];
+      Arrays.setAll((Object[]) array, idx -> sourceArray.get(idx) != null ? sourceArray.get(idx).toString() : null);

Review Comment:
   do you mean Objects::nonNull? As ArrayList..isNull not available



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org