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/07/12 20:43:54 UTC

[GitHub] [arrow] lidavidm opened a new pull request, #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

lidavidm opened a new pull request, #13589:
URL: https://github.com/apache/arrow/pull/13589

   This extends the arrow-jdbc adapter to also allow taking Arrow data and using it to bind JDBC PreparedStatement parameters, allowing you to "round trip" data to a certain extent. This was factored out of arrow-adbc since it's not strictly tied to ADBC. 


-- 
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 #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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

   Benchmark runs are scheduled for baseline = bbf249e056315af0a18d5c0834de9adef117a25f and contender = a5a28377e43b6435eefa5cc684d80d5fb26f387a. a5a28377e43b6435eefa5cc684d80d5fb26f387a is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Failed :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/4a7bd7a1354c45db854ea7d727ca5573...6934a8e8476d4f21baf2a11696de6aff/)
   [Failed :arrow_down:0.34% :arrow_up:0.0%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/90528ac2480945e8922d446d1e108a48...86da1a0e3d854659a4cfa7c85dc614eb/)
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/57e62d33478a428696a2661a3b2b3ff6...4c55abca6fcf4c338d79a7f30f92d277/)
   [Finished :arrow_down:0.36% :arrow_up:0.0%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/f20951c88bc74f9e83ebed0016770369...d5e8d5bd3ce643b8853e48523ad09c8c/)
   Buildkite builds:
   [Failed] [`a5a28377` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/1200)
   [Failed] [`a5a28377` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/1212)
   [Finished] [`a5a28377` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/1194)
   [Finished] [`a5a28377` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/1214)
   [Failed] [`bbf249e0` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/1199)
   [Failed] [`bbf249e0` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/1211)
   [Finished] [`bbf249e0` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/1193)
   [Finished] [`bbf249e0` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/1213)
   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 #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/ColumnBinderArrowTypeVisitor.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.sql.Types;
+import java.time.ZoneId;
+import java.util.Calendar;
+import java.util.TimeZone;
+
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DateMilliVector;
+import org.apache.arrow.vector.Decimal256Vector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.FixedSizeBinaryVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.LargeVarBinaryVector;
+import org.apache.arrow.vector.LargeVarCharVector;
+import org.apache.arrow.vector.SmallIntVector;
+import org.apache.arrow.vector.TimeMicroVector;
+import org.apache.arrow.vector.TimeMilliVector;
+import org.apache.arrow.vector.TimeNanoVector;
+import org.apache.arrow.vector.TimeSecVector;
+import org.apache.arrow.vector.TimeStampVector;
+import org.apache.arrow.vector.TinyIntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+
+/**
+ * Visitor to create the base ColumnBinder for a vector.
+ * <p>
+ * To handle null values, wrap the returned binder in a {@link NullableColumnBinder}.
+ */
+public class ColumnBinderArrowTypeVisitor implements ArrowType.ArrowTypeVisitor<ColumnBinder> {
+  private final FieldVector vector;
+  private final Integer jdbcType;
+
+  /**
+   * Create a binder using a custom JDBC type code.
+   *
+   * @param vector The vector that the binder will wrap.
+   * @param jdbcType The JDBC type code (or null to use the default).
+   */
+  public ColumnBinderArrowTypeVisitor(FieldVector vector, Integer jdbcType) {
+    this.vector = vector;
+    this.jdbcType = jdbcType;
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.Null type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.Struct type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.List type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.LargeList type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.FixedSizeList type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.Union type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.Map type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.Int type) {
+    switch (type.getBitWidth()) {
+      case 8:
+        if (type.getIsSigned()) {
+          return jdbcType == null ? new TinyIntBinder((TinyIntVector) vector) :
+              new TinyIntBinder((TinyIntVector) vector, jdbcType);
+        } else {
+          throw new UnsupportedOperationException(

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 #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcParameterBinder.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.adapter.jdbc.binder.ColumnBinder;
+import org.apache.arrow.util.Preconditions;
+import org.apache.arrow.vector.VectorSchemaRoot;
+
+/**
+ * A binder binds JDBC prepared statement parameters to rows of Arrow data from a VectorSchemaRoot.
+ */
+public class JdbcParameterBinder {
+  private final PreparedStatement statement;
+  private final VectorSchemaRoot root;
+  private final ColumnBinder[] binders;
+  private final int[] parameterIndices;
+  private int nextRowIndex;
+
+  JdbcParameterBinder(
+      final PreparedStatement statement,

Review Comment:
   No, changed it to private, and added some docstrings + an explicit Preconditions check for the last two parameters.



-- 
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] liyafan82 commented on a diff in pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/ColumnBinder.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.sql.PreparedStatement;
+import java.sql.SQLException;
+
+import org.apache.arrow.vector.FieldVector;
+
+/**
+ * A helper to bind values from a wrapped Arrow vector to a JDBC PreparedStatement.
+ */
+public interface ColumnBinder {
+  /**
+   * Bind the given row to the given parameter.
+   *
+   * @param statement The statement to bind to.
+   * @param parameterIndex The parameter to bind to (1-indexed)
+   * @param rowIndex The row to bind values from (0-indexed)
+   * @throws SQLException if an error occurs
+   */
+  void bind(PreparedStatement statement, int parameterIndex, int rowIndex) throws SQLException;
+
+  /**
+   * Get the JDBC type code used by this binder.
+   *
+   * @return A type code from {@link java.sql.Types}.
+   */
+  int getJdbcType();
+
+  /**
+   * Get the vector used by this binder.
+   */
+  FieldVector getVector();
+
+  /**
+   * Create a column binder for a vector (handling nullability).
+   */
+  static ColumnBinder forVector(FieldVector vector) {
+    final ColumnBinder binder = vector.getField().getType().accept(new ColumnBinderArrowTypeVisitor(vector));
+    if (vector.getField().isNullable()) {
+      return new NullableColumnBinder(binder);
+    }
+    return binder;
+  }
+
+  /**
+   * Create a column binder for a vector, overriding the JDBC type code used for null values.
+   */
+  static ColumnBinder forVector(FieldVector vector, int jdbcType) {
+    final ColumnBinder binder = vector.getField().getType().accept(new ColumnBinderArrowTypeVisitor(vector, jdbcType));
+    if (vector.getField().isNullable()) {
+      return new NullableColumnBinder(binder);
+    }
+    return binder;
+  }
+}

Review Comment:
   can we combine the two methods into one? (by using null as jdbcType if a value is not specified)



-- 
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 pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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

   TODOs:
   - [ ] Add documentation
   - [ ] Are we handling time/timestamp types properly when time zones come into play?
   - [ ] Add date support as well


-- 
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] pitrou commented on pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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

   Hmm, I'm out of my depth here, but I guess it looks useful? The main downside being the one-row-at-a-time mechanics, I suppose.


-- 
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 pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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

   I've let this sit for a while so having fixed an additional bug I found, I'll merge this now (though not for 9.0.0)


-- 
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] liyafan82 commented on a diff in pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
docs/source/java/jdbc.rst:
##########
@@ -172,3 +172,101 @@ The JDBC to Arrow type mapping can be obtained at runtime from
 .. _setArraySubTypeByColumnIndexMap: https://arrow.apache.org/docs/java/reference/org/apache/arrow/adapter/jdbc/JdbcToArrowConfigBuilder.html#setArraySubTypeByColumnIndexMap-java.util.Map-
 .. _setArraySubTypeByColumnNameMap: https://arrow.apache.org/docs/java/reference/org/apache/arrow/adapter/jdbc/JdbcToArrowConfigBuilder.html#setArraySubTypeByColumnNameMap-java.util.Map-
 .. _ARROW-17006: https://issues.apache.org/jira/browse/ARROW-17006
+
+VectorSchemaRoot to PreparedStatement Parameter Conversion
+==========================================================
+
+The adapter can bind rows of Arrow data from a VectorSchemaRoot to
+parameters of a JDBC PreparedStatement.  This can be accessed via the
+JdbcParameterBinder class.  Each call to next() will bind parameters
+from the next row of data, and then the application can execute the
+statement, call addBatch(), etc. as desired.  Null values will lead to
+a setNull call with an appropriate JDBC type code (listed below).
+
+.. code-block:: java
+
+   final JdbcParameterBinder binder =
+       JdbcParameterBinder.builder(statement, root).bindAll().build();
+   while (binder.next()) {
+       statement.executeUpdate();
+   }
+   // Use a VectorLoader to update the root
+   binder.reset();
+   while (binder.next()) {
+       statement.executeUpdate();
+   }
+
+The mapping of vectors to parameters, the JDBC type code used by the
+converters, and the type conversions themselves can all be customized:
+
+.. code-block:: java
+
+   final JdbcParameterBinder binder =
+       JdbcParameterBinder.builder(statement, root)
+           .bind(/*parameterIndex*/2, /*columnIndex*/0)
+           .bind(/*parameterIndex*/1, customColumnBinderInstance)
+           .build();
+
+Type Mapping
+------------
+
+The Arrow to JDBC type mapping can be obtained at runtime via
+a method on ColumnBinder.
+
++----------------------------+----------------------------+-------+
+| Arrow Type                 | JDBC Type                  | Notes |
++============================+============================+=======+
+| Binary                     | VARBINARY (setBytes)       |       |
++----------------------------+----------------------------+-------+
+| Bool                       | BOOLEAN (setBoolean)       |       |
++----------------------------+----------------------------+-------+
+| Date32                     | DATE (setDate)             |       |
++----------------------------+----------------------------+-------+
+| Date64                     | DATE (setDate)             |       |
++----------------------------+----------------------------+-------+
+| Decimal128                 | DECIMAL (setBigDecimal)    |       |
++----------------------------+----------------------------+-------+
+| Decimal256                 | DECIMAL (setBigDecimal)    |       |
++----------------------------+----------------------------+-------+
+| FixedSizeBinary            | BINARY (setBytes)          |       |
++----------------------------+----------------------------+-------+
+| Float32                    | REAL (setFloat)            |       |
++----------------------------+----------------------------+-------+
+| Int8                       | TINYINT (setByte)          |       |
++----------------------------+----------------------------+-------+
+| Int16                      | SMALLINT (setShort)        |       |
++----------------------------+----------------------------+-------+
+| Int32                      | INTEGER (setInt)           |       |
++----------------------------+----------------------------+-------+
+| Int64                      | BIGINT (setLong)           |       |
++----------------------------+----------------------------+-------+
+| LargeBinary                | LONGVARBINARY (setBytes)   |       |
++----------------------------+----------------------------+-------+
+| LargeUtf8                  | LONGVARCHAR (setString)    | \(1)  |
++----------------------------+----------------------------+-------+
+| Time[s]                    | TIME (setTime)             |       |
++----------------------------+----------------------------+-------+
+| Time[ms]                   | TIME (setTime)             |       |
++----------------------------+----------------------------+-------+
+| Time[us]                   | TIME (setTime)             |       |
++----------------------------+----------------------------+-------+
+| Time[ns]                   | TIME (setTime)             |       |
++----------------------------+----------------------------+-------+
+| Timestamp[s]               | TIMESTAMP (setTimestamp)   | \(2)  |
++----------------------------+----------------------------+-------+
+| Timestamp[ms]              | TIMESTAMP (setTimestamp)   | \(2)  |
++----------------------------+----------------------------+-------+
+| Timestamp[us]              | TIMESTAMP (setTimestamp)   | \(2)  |
++----------------------------+----------------------------+-------+
+| Timestamp[ns]              | TIMESTAMP (setTimestamp)   | \(2)  |
++----------------------------+----------------------------+-------+
+| Utf8                       | VARCHAR (setString)        |       |
++----------------------------+----------------------------+-------+
+
+* \(1) Strings longer than Integer.MAX_VALUE bytes (the maximum length
+  of a Java ``byte[]``) will cause a runtime exception.
+* \(2) If the timestamp has a timezone, the JDBC type defaults to
+  TIMESTAMP_WITH_TIMEZONE.  If the timestamp has no timezone,

Review Comment:
   what would happen when a timezone is absent, the program would thrown an exception?



-- 
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] liyafan82 commented on a diff in pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcParameterBinder.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.adapter.jdbc.binder.ColumnBinder;
+import org.apache.arrow.util.Preconditions;
+import org.apache.arrow.vector.VectorSchemaRoot;
+
+/**
+ * A binder binds JDBC prepared statement parameters to rows of Arrow data from a VectorSchemaRoot.
+ */
+public class JdbcParameterBinder {
+  private final PreparedStatement statement;
+  private final VectorSchemaRoot root;
+  private final ColumnBinder[] binders;
+  private final int[] parameterIndices;
+  private int nextRowIndex;
+
+  JdbcParameterBinder(
+      final PreparedStatement statement,
+      final VectorSchemaRoot root,
+      final ColumnBinder[] binders,
+      int[] parameterIndices) {
+    this.statement = statement;
+    this.root = root;
+    this.binders = binders;
+    this.parameterIndices = parameterIndices;
+    this.nextRowIndex = 0;
+  }
+
+  /**
+   * Initialize a binder with a builder.
+   *
+   * @param statement The statement to bind to. The binder does not maintain ownership of the statement.
+   * @param root The root to pull data from. The binder does not maintain ownership of the root.

Review Comment:
   "root" -> "vector schema root" or "record batch"?



-- 
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 pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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

   > Interesting work! Thanks. @lidavidm I find an example for `executeUpdate`, does it also support `executeQuery`?
   
   Yes, or really, the only thing this module does is call `setString`, `setInteger`, etc. for you. It's up to the application to then call `executeQuery`, `addBatch`, etc. for maximum flexibility. For instance in ADBC it's used with `addBatch`/`executeBatch`:
   
   https://github.com/apache/arrow-adbc/blob/2485d7c3da217a7190f86128d769a7d0445755ab/java/driver/jdbc/src/main/java/org/apache/arrow/adbc/driver/jdbc/JdbcStatement.java#L160-L166


-- 
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 #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
docs/source/java/jdbc.rst:
##########
@@ -84,8 +84,8 @@ inconsistent scale. A RoundingMode can be set to handle these cases:
      }
    }
 
-Currently, it is not possible to define a custom type conversion for a
-supported or unsupported type.
+Currently, it is not possible to override the type conversion for a
+supported type, or define a new conversion for an unsupported type.

Review Comment:
   What I mean is that you can't implement a custom Consumer and have it be used, all you can do is change what type is assumed by the existing converters. But I'll clarify this



-- 
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 #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


-- 
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 pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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

   CC @toddfarmer @lwhite1 
   
   Not all types are supported here but a core set is. If the approach looks reasonable I can extend coverage to at least Decimal and Binary types.


-- 
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] liyafan82 commented on a diff in pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/BigIntBinder.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Types;
+
+import org.apache.arrow.vector.BigIntVector;
+
+/** A column binder for 8-bit integers. */
+public class BigIntBinder extends BaseColumnBinder<BigIntVector> {
+  public BigIntBinder(BigIntVector vector) {
+    this(vector, Types.BIGINT);
+  }
+
+  public BigIntBinder(BigIntVector vector, int jdbcType) {
+    super(vector, jdbcType);

Review Comment:
   I see. Thanks for the clarification. 



-- 
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] emkornfield commented on a diff in pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcParameterBinder.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.adapter.jdbc.binder.ColumnBinder;
+import org.apache.arrow.util.Preconditions;
+import org.apache.arrow.vector.VectorSchemaRoot;
+
+/**
+ * A binder binds JDBC prepared statement parameters to rows of Arrow data from a VectorSchemaRoot.
+ */
+public class JdbcParameterBinder {
+  private final PreparedStatement statement;
+  private final VectorSchemaRoot root;
+  private final ColumnBinder[] binders;
+  private final int[] parameterIndices;
+  private int nextRowIndex;
+
+  JdbcParameterBinder(
+      final PreparedStatement statement,

Review Comment:
   is this intentionally package private instead of private?  Maybe add a comment on the relationship between the last two parameters?



-- 
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] liyafan82 commented on pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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

   > > Interesting work! Thanks. @lidavidm I find an example for `executeUpdate`, does it also support `executeQuery`?
   > 
   > Yes, or really, the only thing this module does is call `setString`, `setInteger`, etc. for you. It's up to the application to then call `executeQuery`, `addBatch`, etc. for maximum flexibility. For instance in ADBC it's used with `addBatch`/`executeBatch`:
   > 
   > https://github.com/apache/arrow-adbc/blob/2485d7c3da217a7190f86128d769a7d0445755ab/java/driver/jdbc/src/main/java/org/apache/arrow/adbc/driver/jdbc/JdbcStatement.java#L160-L166
   
   Cool! I believe this is a super useful feature. I'd like to review it in the following days.


-- 
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 #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
docs/source/java/jdbc.rst:
##########
@@ -172,3 +172,101 @@ The JDBC to Arrow type mapping can be obtained at runtime from
 .. _setArraySubTypeByColumnIndexMap: https://arrow.apache.org/docs/java/reference/org/apache/arrow/adapter/jdbc/JdbcToArrowConfigBuilder.html#setArraySubTypeByColumnIndexMap-java.util.Map-
 .. _setArraySubTypeByColumnNameMap: https://arrow.apache.org/docs/java/reference/org/apache/arrow/adapter/jdbc/JdbcToArrowConfigBuilder.html#setArraySubTypeByColumnNameMap-java.util.Map-
 .. _ARROW-17006: https://issues.apache.org/jira/browse/ARROW-17006
+
+VectorSchemaRoot to PreparedStatement Parameter Conversion
+==========================================================
+
+The adapter can bind rows of Arrow data from a VectorSchemaRoot to
+parameters of a JDBC PreparedStatement.  This can be accessed via the
+JdbcParameterBinder class.  Each call to next() will bind parameters
+from the next row of data, and then the application can execute the
+statement, call addBatch(), etc. as desired.  Null values will lead to
+a setNull call with an appropriate JDBC type code (listed below).
+
+.. code-block:: java
+
+   final JdbcParameterBinder binder =
+       JdbcParameterBinder.builder(statement, root).bindAll().build();
+   while (binder.next()) {
+       statement.executeUpdate();
+   }
+   // Use a VectorLoader to update the root
+   binder.reset();
+   while (binder.next()) {
+       statement.executeUpdate();
+   }
+
+The mapping of vectors to parameters, the JDBC type code used by the
+converters, and the type conversions themselves can all be customized:
+
+.. code-block:: java
+
+   final JdbcParameterBinder binder =
+       JdbcParameterBinder.builder(statement, root)
+           .bind(/*parameterIndex*/2, /*columnIndex*/0)
+           .bind(/*parameterIndex*/1, customColumnBinderInstance)
+           .build();
+
+Type Mapping
+------------
+
+The Arrow to JDBC type mapping can be obtained at runtime via
+a method on ColumnBinder.
+
++----------------------------+----------------------------+-------+
+| Arrow Type                 | JDBC Type                  | Notes |
++============================+============================+=======+
+| Binary                     | VARBINARY (setBytes)       |       |
++----------------------------+----------------------------+-------+
+| Bool                       | BOOLEAN (setBoolean)       |       |
++----------------------------+----------------------------+-------+
+| Date32                     | DATE (setDate)             |       |
++----------------------------+----------------------------+-------+
+| Date64                     | DATE (setDate)             |       |
++----------------------------+----------------------------+-------+
+| Decimal128                 | DECIMAL (setBigDecimal)    |       |
++----------------------------+----------------------------+-------+
+| Decimal256                 | DECIMAL (setBigDecimal)    |       |
++----------------------------+----------------------------+-------+
+| FixedSizeBinary            | BINARY (setBytes)          |       |
++----------------------------+----------------------------+-------+
+| Float32                    | REAL (setFloat)            |       |
++----------------------------+----------------------------+-------+
+| Int8                       | TINYINT (setByte)          |       |
++----------------------------+----------------------------+-------+
+| Int16                      | SMALLINT (setShort)        |       |
++----------------------------+----------------------------+-------+
+| Int32                      | INTEGER (setInt)           |       |
++----------------------------+----------------------------+-------+
+| Int64                      | BIGINT (setLong)           |       |
++----------------------------+----------------------------+-------+
+| LargeBinary                | LONGVARBINARY (setBytes)   |       |
++----------------------------+----------------------------+-------+
+| LargeUtf8                  | LONGVARCHAR (setString)    | \(1)  |
++----------------------------+----------------------------+-------+
+| Time[s]                    | TIME (setTime)             |       |
++----------------------------+----------------------------+-------+
+| Time[ms]                   | TIME (setTime)             |       |
++----------------------------+----------------------------+-------+
+| Time[us]                   | TIME (setTime)             |       |
++----------------------------+----------------------------+-------+
+| Time[ns]                   | TIME (setTime)             |       |
++----------------------------+----------------------------+-------+
+| Timestamp[s]               | TIMESTAMP (setTimestamp)   | \(2)  |
++----------------------------+----------------------------+-------+
+| Timestamp[ms]              | TIMESTAMP (setTimestamp)   | \(2)  |
++----------------------------+----------------------------+-------+
+| Timestamp[us]              | TIMESTAMP (setTimestamp)   | \(2)  |
++----------------------------+----------------------------+-------+
+| Timestamp[ns]              | TIMESTAMP (setTimestamp)   | \(2)  |
++----------------------------+----------------------------+-------+
+| Utf8                       | VARCHAR (setString)        |       |
++----------------------------+----------------------------+-------+
+
+* \(1) Strings longer than Integer.MAX_VALUE bytes (the maximum length
+  of a Java ``byte[]``) will cause a runtime exception.
+* \(2) If the timestamp has a timezone, the JDBC type defaults to
+  TIMESTAMP_WITH_TIMEZONE.  If the timestamp has no timezone,

Review Comment:
   It'll just call `setTimestamp(int, Timestamp)` instead of `setTimestamp(int, Timestamp, Calendar)`, I'll update the doc



-- 
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] lwhite1 commented on pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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

   Overall, this looks really nice to me. 
   
   One minor nit (not from the current change set): The statement on line 87 "Currently, it is not possible to define a custom type conversion for a supported or unsupported type."  has me scratching my head a bit  Should it just say "it is not possible to define a custom type conversion"?  If there's a quick re-phrasing that helps, it might be worth adding.  
   


-- 
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] liyafan82 commented on pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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

   Interesting work! Thanks. @lidavidm 
   I find an example for `executeUpdate`, does it also support `executeQuery`?


-- 
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] liyafan82 commented on a diff in pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
docs/source/java/jdbc.rst:
##########
@@ -84,8 +84,8 @@ inconsistent scale. A RoundingMode can be set to handle these cases:
      }
    }
 
-Currently, it is not possible to define a custom type conversion for a
-supported or unsupported type.
+Currently, it is not possible to override the type conversion for a
+supported type, or define a new conversion for an unsupported type.

Review Comment:
   Thanks.



##########
docs/source/java/jdbc.rst:
##########
@@ -172,3 +172,101 @@ The JDBC to Arrow type mapping can be obtained at runtime from
 .. _setArraySubTypeByColumnIndexMap: https://arrow.apache.org/docs/java/reference/org/apache/arrow/adapter/jdbc/JdbcToArrowConfigBuilder.html#setArraySubTypeByColumnIndexMap-java.util.Map-
 .. _setArraySubTypeByColumnNameMap: https://arrow.apache.org/docs/java/reference/org/apache/arrow/adapter/jdbc/JdbcToArrowConfigBuilder.html#setArraySubTypeByColumnNameMap-java.util.Map-
 .. _ARROW-17006: https://issues.apache.org/jira/browse/ARROW-17006
+
+VectorSchemaRoot to PreparedStatement Parameter Conversion
+==========================================================
+
+The adapter can bind rows of Arrow data from a VectorSchemaRoot to
+parameters of a JDBC PreparedStatement.  This can be accessed via the
+JdbcParameterBinder class.  Each call to next() will bind parameters
+from the next row of data, and then the application can execute the
+statement, call addBatch(), etc. as desired.  Null values will lead to
+a setNull call with an appropriate JDBC type code (listed below).
+
+.. code-block:: java
+
+   final JdbcParameterBinder binder =
+       JdbcParameterBinder.builder(statement, root).bindAll().build();
+   while (binder.next()) {
+       statement.executeUpdate();
+   }
+   // Use a VectorLoader to update the root
+   binder.reset();
+   while (binder.next()) {
+       statement.executeUpdate();
+   }
+
+The mapping of vectors to parameters, the JDBC type code used by the
+converters, and the type conversions themselves can all be customized:
+
+.. code-block:: java
+
+   final JdbcParameterBinder binder =
+       JdbcParameterBinder.builder(statement, root)
+           .bind(/*parameterIndex*/2, /*columnIndex*/0)
+           .bind(/*parameterIndex*/1, customColumnBinderInstance)
+           .build();
+
+Type Mapping
+------------
+
+The Arrow to JDBC type mapping can be obtained at runtime via
+a method on ColumnBinder.
+
++----------------------------+----------------------------+-------+
+| Arrow Type                 | JDBC Type                  | Notes |
++============================+============================+=======+
+| Binary                     | VARBINARY (setBytes)       |       |
++----------------------------+----------------------------+-------+
+| Bool                       | BOOLEAN (setBoolean)       |       |
++----------------------------+----------------------------+-------+
+| Date32                     | DATE (setDate)             |       |
++----------------------------+----------------------------+-------+
+| Date64                     | DATE (setDate)             |       |
++----------------------------+----------------------------+-------+
+| Decimal128                 | DECIMAL (setBigDecimal)    |       |
++----------------------------+----------------------------+-------+
+| Decimal256                 | DECIMAL (setBigDecimal)    |       |
++----------------------------+----------------------------+-------+
+| FixedSizeBinary            | BINARY (setBytes)          |       |
++----------------------------+----------------------------+-------+
+| Float32                    | REAL (setFloat)            |       |
++----------------------------+----------------------------+-------+
+| Int8                       | TINYINT (setByte)          |       |
++----------------------------+----------------------------+-------+
+| Int16                      | SMALLINT (setShort)        |       |
++----------------------------+----------------------------+-------+
+| Int32                      | INTEGER (setInt)           |       |
++----------------------------+----------------------------+-------+
+| Int64                      | BIGINT (setLong)           |       |
++----------------------------+----------------------------+-------+
+| LargeBinary                | LONGVARBINARY (setBytes)   |       |
++----------------------------+----------------------------+-------+
+| LargeUtf8                  | LONGVARCHAR (setString)    | \(1)  |
++----------------------------+----------------------------+-------+
+| Time[s]                    | TIME (setTime)             |       |
++----------------------------+----------------------------+-------+
+| Time[ms]                   | TIME (setTime)             |       |
++----------------------------+----------------------------+-------+
+| Time[us]                   | TIME (setTime)             |       |
++----------------------------+----------------------------+-------+
+| Time[ns]                   | TIME (setTime)             |       |
++----------------------------+----------------------------+-------+
+| Timestamp[s]               | TIMESTAMP (setTimestamp)   | \(2)  |
++----------------------------+----------------------------+-------+
+| Timestamp[ms]              | TIMESTAMP (setTimestamp)   | \(2)  |
++----------------------------+----------------------------+-------+
+| Timestamp[us]              | TIMESTAMP (setTimestamp)   | \(2)  |
++----------------------------+----------------------------+-------+
+| Timestamp[ns]              | TIMESTAMP (setTimestamp)   | \(2)  |
++----------------------------+----------------------------+-------+
+| Utf8                       | VARCHAR (setString)        |       |
++----------------------------+----------------------------+-------+
+
+* \(1) Strings longer than Integer.MAX_VALUE bytes (the maximum length
+  of a Java ``byte[]``) will cause a runtime exception.
+* \(2) If the timestamp has a timezone, the JDBC type defaults to
+  TIMESTAMP_WITH_TIMEZONE.  If the timestamp has no timezone,

Review Comment:
   Thanks for the clarification.



-- 
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] liyafan82 commented on a diff in pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/ColumnBinderArrowTypeVisitor.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.sql.Types;
+import java.time.ZoneId;
+import java.util.Calendar;
+import java.util.TimeZone;
+
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DateMilliVector;
+import org.apache.arrow.vector.Decimal256Vector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.FixedSizeBinaryVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.LargeVarBinaryVector;
+import org.apache.arrow.vector.LargeVarCharVector;
+import org.apache.arrow.vector.SmallIntVector;
+import org.apache.arrow.vector.TimeMicroVector;
+import org.apache.arrow.vector.TimeMilliVector;
+import org.apache.arrow.vector.TimeNanoVector;
+import org.apache.arrow.vector.TimeSecVector;
+import org.apache.arrow.vector.TimeStampVector;
+import org.apache.arrow.vector.TinyIntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+
+/**
+ * Visitor to create the base ColumnBinder for a vector.
+ * <p>
+ * To handle null values, wrap the returned binder in a {@link NullableColumnBinder}.
+ */
+public class ColumnBinderArrowTypeVisitor implements ArrowType.ArrowTypeVisitor<ColumnBinder> {
+  private final FieldVector vector;
+  private final Integer jdbcType;
+
+  /**
+   * Create a binder using a custom JDBC type code.
+   *
+   * @param vector The vector that the binder will wrap.
+   * @param jdbcType The JDBC type code (or null to use the default).
+   */
+  public ColumnBinderArrowTypeVisitor(FieldVector vector, Integer jdbcType) {
+    this.vector = vector;
+    this.jdbcType = jdbcType;
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.Null type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.Struct type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.List type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.LargeList type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.FixedSizeList type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.Union type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.Map type) {
+    throw new UnsupportedOperationException("No column binder implemented for type " + type);
+  }
+
+  @Override
+  public ColumnBinder visit(ArrowType.Int type) {
+    switch (type.getBitWidth()) {
+      case 8:
+        if (type.getIsSigned()) {
+          return jdbcType == null ? new TinyIntBinder((TinyIntVector) vector) :
+              new TinyIntBinder((TinyIntVector) vector, jdbcType);
+        } else {
+          throw new UnsupportedOperationException(

Review Comment:
   Maybe we can extract this statement for all type widths, at the beginning of this method?



-- 
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] liyafan82 commented on a diff in pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/BaseColumnBinder.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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 org.apache.arrow.vector.FieldVector;
+
+/**
+ * Base class for ColumnBinder implementations.
+ * @param <V> The concrete FieldVector subtype.
+ */
+public abstract class BaseColumnBinder<V extends FieldVector> implements ColumnBinder {
+  protected V vector;
+  protected int jdbcType;

Review Comment:
   can we declare the fields as `final`?



-- 
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] liyafan82 commented on a diff in pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
docs/source/java/jdbc.rst:
##########
@@ -84,8 +84,8 @@ inconsistent scale. A RoundingMode can be set to handle these cases:
      }
    }
 
-Currently, it is not possible to define a custom type conversion for a
-supported or unsupported type.
+Currently, it is not possible to override the type conversion for a
+supported type, or define a new conversion for an unsupported type.

Review Comment:
   Sorry, I think we do support overriding the conversion now. Please see https://github.com/apache/arrow/blob/master/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java#L76



-- 
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] liyafan82 commented on a diff in pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/BigIntBinder.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Types;
+
+import org.apache.arrow.vector.BigIntVector;
+
+/** A column binder for 8-bit integers. */
+public class BigIntBinder extends BaseColumnBinder<BigIntVector> {
+  public BigIntBinder(BigIntVector vector) {
+    this(vector, Types.BIGINT);
+  }
+
+  public BigIntBinder(BigIntVector vector, int jdbcType) {
+    super(vector, jdbcType);

Review Comment:
   Is a type other than `Types.BIGINT` allowed 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] lidavidm commented on pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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

   Thanks for taking a look!
   
   Updated the docs, and implemented binders for binary types and decimals.


-- 
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 pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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

   @emkornfield @liyafan82 @pitrou any opinions on having this functionality (binding Arrow data -> JDBC prepared statement parameters) 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] github-actions[bot] commented on pull request #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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

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


-- 
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 #13589: ARROW-17004: [Java] Add utility to bind Arrow data to JDBC parameters

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


##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/BigIntBinder.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Types;
+
+import org.apache.arrow.vector.BigIntVector;
+
+/** A column binder for 8-bit integers. */
+public class BigIntBinder extends BaseColumnBinder<BigIntVector> {
+  public BigIntBinder(BigIntVector vector) {
+    this(vector, Types.BIGINT);
+  }
+
+  public BigIntBinder(BigIntVector vector, int jdbcType) {
+    super(vector, jdbcType);

Review Comment:
   In principle, I wanted to allow things like binding an Int64 vector to an Int field, maybe that is too much flexibility though.



##########
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/binder/BaseColumnBinder.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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 org.apache.arrow.vector.FieldVector;
+
+/**
+ * Base class for ColumnBinder implementations.
+ * @param <V> The concrete FieldVector subtype.
+ */
+public abstract class BaseColumnBinder<V extends FieldVector> implements ColumnBinder {
+  protected V vector;
+  protected int jdbcType;

Review Comment:
   Done, thanks for catching that.



-- 
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