, AutoCloseable {
-
- private final ResultSet resultSet;
- private final JdbcToArrowConfig config;
-
- private final Schema schema;
- private final ResultSetMetaData rsmd;
-
- private final JdbcConsumer[] consumers;
- final CompositeJdbcConsumer compositeConsumer;
-
- // this is used only if resuing vector schema root is enabled.
- private VectorSchemaRoot nextBatch;
-
- private final int targetBatchSize;
-
- // This is used to track whether the ResultSet has been fully read, and is needed specifically for
- // cases where there
- // is a ResultSet having zero rows (empty):
- private boolean readComplete = false;
-
- /** Construct an instance. */
- private ArrowVectorIterator(ResultSet resultSet, JdbcToArrowConfig config) throws SQLException {
- this.resultSet = resultSet;
- this.config = config;
- this.schema = JdbcToArrowUtils.jdbcToArrowSchema(resultSet.getMetaData(), config);
- this.targetBatchSize = config.getTargetBatchSize();
-
- rsmd = resultSet.getMetaData();
- consumers = new JdbcConsumer[rsmd.getColumnCount()];
- this.compositeConsumer = new CompositeJdbcConsumer(consumers);
- this.nextBatch = config.isReuseVectorSchemaRoot() ? createVectorSchemaRoot() : null;
- }
-
- /** Create a ArrowVectorIterator to partially convert data. */
- public static ArrowVectorIterator create(ResultSet resultSet, JdbcToArrowConfig config)
- throws SQLException {
- ArrowVectorIterator iterator = null;
- try {
- iterator = new ArrowVectorIterator(resultSet, config);
- } catch (Throwable e) {
- AutoCloseables.close(e, iterator);
- throw new RuntimeException("Error occurred while creating iterator.", e);
- }
- return iterator;
- }
-
- private void consumeData(VectorSchemaRoot root) {
- // consume data
- try {
- int readRowCount = 0;
- if (targetBatchSize == JdbcToArrowConfig.NO_LIMIT_BATCH_SIZE) {
- while (resultSet.next()) {
- ValueVectorUtility.ensureCapacity(root, readRowCount + 1);
- compositeConsumer.consume(resultSet);
- readRowCount++;
- }
- readComplete = true;
- } else {
- while ((readRowCount < targetBatchSize) && !readComplete) {
- if (resultSet.next()) {
- compositeConsumer.consume(resultSet);
- readRowCount++;
- } else {
- readComplete = true;
- }
- }
- }
-
- root.setRowCount(readRowCount);
- } catch (Throwable e) {
- compositeConsumer.close();
- if (e instanceof JdbcConsumerException) {
- throw (JdbcConsumerException) e;
- } else {
- throw new RuntimeException("Error occurred while consuming data.", e);
- }
- }
- }
-
- private VectorSchemaRoot createVectorSchemaRoot() throws SQLException {
- VectorSchemaRoot root = null;
- try {
- root = VectorSchemaRoot.create(schema, config.getAllocator());
- if (config.getTargetBatchSize() != JdbcToArrowConfig.NO_LIMIT_BATCH_SIZE) {
- ValueVectorUtility.preAllocate(root, config.getTargetBatchSize());
- }
- } catch (Throwable e) {
- if (root != null) {
- root.close();
- }
- throw new RuntimeException("Error occurred while creating schema root.", e);
- }
- initialize(root);
- return root;
- }
-
- private void initialize(VectorSchemaRoot root) throws SQLException {
- for (int i = 1; i <= consumers.length; i++) {
- final JdbcFieldInfo columnFieldInfo =
- JdbcToArrowUtils.getJdbcFieldInfoForColumn(rsmd, i, config);
- ArrowType arrowType = config.getJdbcToArrowTypeConverter().apply(columnFieldInfo);
- consumers[i - 1] =
- config
- .getJdbcConsumerGetter()
- .apply(
- arrowType,
- i,
- isColumnNullable(resultSet.getMetaData(), i, columnFieldInfo),
- root.getVector(i - 1),
- config);
- }
- }
-
- // Loads the next schema root or null if no more rows are available.
- private void load(VectorSchemaRoot root) {
- for (int i = 0; i < consumers.length; i++) {
- FieldVector vec = root.getVector(i);
- if (config.isReuseVectorSchemaRoot()) {
- // if we are reusing the vector schema root,
- // we must reset the vector before populating it with data.
- vec.reset();
- }
- consumers[i].resetValueVector(vec);
- }
-
- consumeData(root);
- }
-
- @Override
- public boolean hasNext() {
- return !readComplete;
- }
-
- /**
- * Gets the next vector. If {@link JdbcToArrowConfig#isReuseVectorSchemaRoot()} is false, the
- * client is responsible for freeing its resources.
- *
- * @throws JdbcConsumerException on error from VectorConsumer
- */
- @Override
- public VectorSchemaRoot next() {
- Preconditions.checkArgument(hasNext());
- try {
- VectorSchemaRoot ret =
- config.isReuseVectorSchemaRoot() ? nextBatch : createVectorSchemaRoot();
- load(ret);
- return ret;
- } catch (Exception e) {
- close();
- if (e instanceof JdbcConsumerException) {
- throw (JdbcConsumerException) e;
- } else {
- throw new RuntimeException("Error occurred while getting next schema root.", e);
- }
- }
- }
-
- /**
- * Clean up resources ONLY WHEN THE {@link VectorSchemaRoot} HOLDING EACH BATCH IS REUSED. If a
- * new VectorSchemaRoot is created for each batch, each root must be closed manually by the client
- * code.
- */
- @Override
- public void close() {
- if (config.isReuseVectorSchemaRoot()) {
- nextBatch.close();
- compositeConsumer.close();
- }
- }
-}
diff --git a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/Constants.java b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/Constants.java
deleted file mode 100644
index 30e734a68d511..0000000000000
--- a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/Constants.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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;
-
-/** String constants used for metadata returned on Vectors. */
-public class Constants {
- private Constants() {}
-
- public static final String SQL_CATALOG_NAME_KEY = "SQL_CATALOG_NAME";
- public static final String SQL_SCHEMA_NAME_KEY = "SQL_SCHEMA_NAME";
- public static final String SQL_TABLE_NAME_KEY = "SQL_TABLE_NAME";
- public static final String SQL_COLUMN_NAME_KEY = "SQL_COLUMN_NAME";
- public static final String SQL_TYPE_KEY = "SQL_TYPE";
-}
diff --git a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcFieldInfo.java b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcFieldInfo.java
deleted file mode 100644
index 6becac0bbc10c..0000000000000
--- a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcFieldInfo.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * 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.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Types;
-import org.apache.arrow.util.Preconditions;
-
-/**
- * This class represents the information about a JDBC ResultSet Field that is needed to construct an
- * {@link org.apache.arrow.vector.types.pojo.ArrowType}. Currently, this is:
- *
- *
- * - The JDBC {@link java.sql.Types} type.
- *
- The nullability.
- *
- The field's precision (used for {@link java.sql.Types#DECIMAL} and {@link
- * java.sql.Types#NUMERIC} types).
- *
- The field's scale (used for {@link java.sql.Types#DECIMAL} and {@link
- * java.sql.Types#NUMERIC} types).
- *
- */
-public class JdbcFieldInfo {
- private final int column;
- private final int jdbcType;
- private final int nullability;
- private final int precision;
- private final int scale;
- private final String typeName;
- private final int displaySize;
-
- /**
- * Builds a JdbcFieldInfo
using only the {@link java.sql.Types} type. Do not use this
- * constructor if the field type is {@link java.sql.Types#DECIMAL} or {@link
- * java.sql.Types#NUMERIC}; the precision and scale will be set to 0
.
- *
- * @param jdbcType The {@link java.sql.Types} type.
- * @throws IllegalArgumentException if jdbcType is {@link java.sql.Types#DECIMAL} or {@link
- * java.sql.Types#NUMERIC}.
- */
- public JdbcFieldInfo(int jdbcType) {
- Preconditions.checkArgument(
- (jdbcType != Types.DECIMAL && jdbcType != Types.NUMERIC),
- "DECIMAL and NUMERIC types require a precision and scale; please use another constructor.");
- this.column = 0;
- this.jdbcType = jdbcType;
- this.nullability = ResultSetMetaData.columnNullableUnknown;
- this.precision = 0;
- this.scale = 0;
- this.typeName = "";
- this.displaySize = 0;
- }
-
- /**
- * Builds a JdbcFieldInfo
from the {@link java.sql.Types} type, precision, and scale.
- * Use this constructor for {@link java.sql.Types#DECIMAL} and {@link java.sql.Types#NUMERIC}
- * types.
- *
- * @param jdbcType The {@link java.sql.Types} type.
- * @param precision The field's numeric precision.
- * @param scale The field's numeric scale.
- */
- public JdbcFieldInfo(int jdbcType, int precision, int scale) {
- this.column = 0;
- this.jdbcType = jdbcType;
- this.nullability = ResultSetMetaData.columnNullableUnknown;
- this.precision = precision;
- this.scale = scale;
- this.typeName = "";
- this.displaySize = 0;
- }
-
- /**
- * Builds a JdbcFieldInfo
from the {@link java.sql.Types} type, nullability,
- * precision, and scale.
- *
- * @param jdbcType The {@link java.sql.Types} type.
- * @param nullability The nullability. Must be one of {@link ResultSetMetaData#columnNoNulls},
- * {@link ResultSetMetaData#columnNullable}, or {@link
- * ResultSetMetaData#columnNullableUnknown}.
- * @param precision The field's numeric precision.
- * @param scale The field's numeric scale.
- */
- public JdbcFieldInfo(int jdbcType, int nullability, int precision, int scale) {
- this.column = 0;
- this.jdbcType = jdbcType;
- this.nullability = nullability;
- this.precision = precision;
- this.scale = scale;
- this.typeName = "";
- this.displaySize = 0;
- }
-
- /**
- * Builds a JdbcFieldInfo
from the corresponding {@link java.sql.ResultSetMetaData}
- * column.
- *
- * @param rsmd The {@link java.sql.ResultSetMetaData} to get the field information from.
- * @param column The column to get the field information for (on a 1-based index).
- * @throws SQLException If the column information cannot be retrieved.
- * @throws NullPointerException if rsmd
is null
.
- * @throws IllegalArgumentException if column
is out of bounds.
- */
- public JdbcFieldInfo(ResultSetMetaData rsmd, int column) throws SQLException {
- Preconditions.checkNotNull(rsmd, "ResultSetMetaData cannot be null.");
- Preconditions.checkArgument(
- column > 0, "ResultSetMetaData columns have indices starting at 1.");
- Preconditions.checkArgument(
- column <= rsmd.getColumnCount(),
- "The index must be within the number of columns (1 to %s, inclusive)",
- rsmd.getColumnCount());
-
- this.column = column;
- this.jdbcType = rsmd.getColumnType(column);
- this.nullability = rsmd.isNullable(column);
- this.precision = rsmd.getPrecision(column);
- this.scale = rsmd.getScale(column);
- this.typeName = rsmd.getColumnTypeName(column);
- this.displaySize = rsmd.getColumnDisplaySize(column);
- }
-
- /**
- * Builds a JdbcFieldInfo
from the corresponding row from a {@link
- * java.sql.DatabaseMetaData#getColumns} ResultSet.
- *
- * @param rs The {@link java.sql.ResultSet} to get the field information from.
- * @throws SQLException If the column information cannot be retrieved.
- */
- public JdbcFieldInfo(ResultSet rs) throws SQLException {
- this.column = rs.getInt("ORDINAL_POSITION");
- this.jdbcType = rs.getInt("DATA_TYPE");
- this.nullability = rs.getInt("NULLABLE");
- this.precision = rs.getInt("COLUMN_SIZE");
- this.scale = rs.getInt("DECIMAL_DIGITS");
- this.typeName = rs.getString("TYPE_NAME");
- this.displaySize = rs.getInt("CHAR_OCTET_LENGTH");
- }
-
- /** The {@link java.sql.Types} type. */
- public int getJdbcType() {
- return jdbcType;
- }
-
- /** The nullability. */
- public int isNullable() {
- return nullability;
- }
-
- /**
- * The numeric precision, for {@link java.sql.Types#NUMERIC} and {@link java.sql.Types#DECIMAL}
- * types.
- */
- public int getPrecision() {
- return precision;
- }
-
- /**
- * The numeric scale, for {@link java.sql.Types#NUMERIC} and {@link java.sql.Types#DECIMAL} types.
- */
- public int getScale() {
- return scale;
- }
-
- /** The column index for query column. */
- public int getColumn() {
- return column;
- }
-
- /** The type name as reported by the database. */
- public String getTypeName() {
- return typeName;
- }
-
- /** The max number of characters for the column. */
- public int getDisplaySize() {
- return displaySize;
- }
-}
diff --git a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcParameterBinder.java b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcParameterBinder.java
deleted file mode 100644
index fd4721bcd9c4e..0000000000000
--- a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcParameterBinder.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * 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.
- *
- * Each row of the VectorSchemaRoot will be bound to the configured parameters of the
- * PreparedStatement. One row of data is bound at a time.
- */
-public class JdbcParameterBinder {
- private final PreparedStatement statement;
- private final VectorSchemaRoot root;
- private final ColumnBinder[] binders;
- private final int[] parameterIndices;
- private int nextRowIndex;
-
- /**
- * Create a new parameter binder.
- *
- * @param statement The statement to bind parameters to.
- * @param root The VectorSchemaRoot to pull data from.
- * @param binders Column binders to translate from Arrow data to JDBC parameters, one per
- * parameter.
- * @param parameterIndices For each binder in binders, the index of the parameter to bind
- * to.
- */
- private JdbcParameterBinder(
- final PreparedStatement statement,
- final VectorSchemaRoot root,
- final ColumnBinder[] binders,
- int[] parameterIndices) {
- Preconditions.checkArgument(
- binders.length == parameterIndices.length,
- "Number of column binders (%s) must equal number of parameter indices (%s)",
- binders.length,
- parameterIndices.length);
- 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 {@link VectorSchemaRoot} to pull data from. The binder does not maintain
- * ownership of the vector schema root.
- */
- public static Builder builder(final PreparedStatement statement, final VectorSchemaRoot root) {
- return new Builder(statement, root);
- }
-
- /** Reset the binder (so the root can be updated with new data). */
- public void reset() {
- nextRowIndex = 0;
- }
-
- /**
- * Bind the next row of data to the parameters of the statement.
- *
- *
After this, the application should call the desired method on the prepared statement, such
- * as {@link PreparedStatement#executeUpdate()}, or {@link PreparedStatement#addBatch()}.
- *
- * @return true if a row was bound, false if rows were exhausted
- */
- public boolean next() throws SQLException {
- if (nextRowIndex >= root.getRowCount()) {
- return false;
- }
- for (int i = 0; i < parameterIndices.length; i++) {
- final int parameterIndex = parameterIndices[i];
- binders[i].bind(statement, parameterIndex, nextRowIndex);
- }
- nextRowIndex++;
- return true;
- }
-
- /** A builder for a {@link JdbcParameterBinder}. */
- public static class Builder {
- private final PreparedStatement statement;
- private final VectorSchemaRoot root;
- private final Map bindings;
-
- Builder(PreparedStatement statement, VectorSchemaRoot root) {
- this.statement = statement;
- this.root = root;
- this.bindings = new HashMap<>();
- }
-
- /** Bind each column to the corresponding parameter in order. */
- public Builder bindAll() {
- for (int i = 0; i < root.getFieldVectors().size(); i++) {
- bind(/*parameterIndex=*/ i + 1, /*columnIndex=*/ i);
- }
- return this;
- }
-
- /** Bind the given parameter to the given column using the default binder. */
- public Builder bind(int parameterIndex, int columnIndex) {
- return bind(parameterIndex, ColumnBinder.forVector(root.getVector(columnIndex)));
- }
-
- /** Bind the given parameter using the given binder. */
- public Builder bind(int parameterIndex, ColumnBinder binder) {
- Preconditions.checkArgument(
- parameterIndex > 0, "parameterIndex %d must be positive", parameterIndex);
- bindings.put(parameterIndex, binder);
- return this;
- }
-
- /** Build the binder. */
- public JdbcParameterBinder build() {
- ColumnBinder[] binders = new ColumnBinder[bindings.size()];
- int[] parameterIndices = new int[bindings.size()];
- int index = 0;
- for (Map.Entry entry : bindings.entrySet()) {
- binders[index] = entry.getValue();
- parameterIndices[index] = entry.getKey();
- index++;
- }
- return new JdbcParameterBinder(statement, root, binders, parameterIndices);
- }
- }
-}
diff --git a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrow.java b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrow.java
deleted file mode 100644
index 493e53056f945..0000000000000
--- a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrow.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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.io.IOException;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.util.Preconditions;
-
-/**
- * Utility class to convert JDBC objects to columnar Arrow format objects.
- *
- * This utility uses following data mapping to map JDBC/SQL datatype to Arrow data types.
- *
- *
CHAR --> ArrowType.Utf8 NCHAR --> ArrowType.Utf8 VARCHAR --> ArrowType.Utf8 NVARCHAR -->
- * ArrowType.Utf8 LONGVARCHAR --> ArrowType.Utf8 LONGNVARCHAR --> ArrowType.Utf8 NUMERIC -->
- * ArrowType.Decimal(precision, scale) DECIMAL --> ArrowType.Decimal(precision, scale) BIT -->
- * ArrowType.Bool TINYINT --> ArrowType.Int(8, signed) SMALLINT --> ArrowType.Int(16, signed)
- * INTEGER --> ArrowType.Int(32, signed) BIGINT --> ArrowType.Int(64, signed) REAL -->
- * ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE) FLOAT -->
- * ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE) DOUBLE -->
- * ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE) BINARY --> ArrowType.Binary VARBINARY -->
- * ArrowType.Binary LONGVARBINARY --> ArrowType.Binary DATE --> ArrowType.Date(DateUnit.MILLISECOND)
- * TIME --> ArrowType.Time(TimeUnit.MILLISECOND, 32) TIMESTAMP -->
- * ArrowType.Timestamp(TimeUnit.MILLISECOND, timezone=null) CLOB --> ArrowType.Utf8 BLOB -->
- * ArrowType.Binary
- *
- * @since 0.10.0
- */
-public class JdbcToArrow {
-
- /*----------------------------------------------------------------*
- | |
- | Partial Convert API |
- | |
- *----------------------------------------------------------------*/
-
- /**
- * For the given JDBC {@link ResultSet}, fetch the data from Relational DB and convert it to Arrow
- * objects. Note here uses the default targetBatchSize = 1024.
- *
- * @param resultSet ResultSet to use to fetch the data from underlying database
- * @param allocator Memory allocator
- * @return Arrow Data Objects {@link ArrowVectorIterator}
- * @throws SQLException on error
- */
- public static ArrowVectorIterator sqlToArrowVectorIterator(
- ResultSet resultSet, BufferAllocator allocator) throws SQLException, IOException {
- Preconditions.checkNotNull(allocator, "Memory Allocator object cannot be null");
-
- JdbcToArrowConfig config = new JdbcToArrowConfig(allocator, JdbcToArrowUtils.getUtcCalendar());
- return sqlToArrowVectorIterator(resultSet, config);
- }
-
- /**
- * For the given JDBC {@link ResultSet}, fetch the data from Relational DB and convert it to Arrow
- * objects. Note if not specify {@link JdbcToArrowConfig#targetBatchSize}, will use default value
- * 1024.
- *
- * @param resultSet ResultSet to use to fetch the data from underlying database
- * @param config Configuration of the conversion from JDBC to Arrow.
- * @return Arrow Data Objects {@link ArrowVectorIterator}
- * @throws SQLException on error
- */
- public static ArrowVectorIterator sqlToArrowVectorIterator(
- ResultSet resultSet, JdbcToArrowConfig config) throws SQLException, IOException {
- Preconditions.checkNotNull(resultSet, "JDBC ResultSet object cannot be null");
- Preconditions.checkNotNull(config, "The configuration cannot be null");
- return ArrowVectorIterator.create(resultSet, config);
- }
-}
diff --git a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java
deleted file mode 100644
index 1bfcfc8fe00aa..0000000000000
--- a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java
+++ /dev/null
@@ -1,405 +0,0 @@
-/*
- * 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.math.RoundingMode;
-import java.util.Calendar;
-import java.util.Map;
-import java.util.function.Function;
-import org.apache.arrow.adapter.jdbc.consumer.JdbcConsumer;
-import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.util.Preconditions;
-import org.apache.arrow.vector.FieldVector;
-import org.apache.arrow.vector.types.pojo.ArrowType;
-
-/**
- * This class configures the JDBC-to-Arrow conversion process.
- *
- *
The allocator is used to construct the {@link org.apache.arrow.vector.VectorSchemaRoot}, and
- * the calendar is used to define the time zone of any {@link
- * org.apache.arrow.vector.types.pojo.ArrowType.Timestamp} fields that are created during the
- * conversion. Neither field may be null
.
- *
- *
If the includeMetadata
flag is set, the Arrow field metadata will contain
- * information from the corresponding {@link java.sql.ResultSetMetaData} that was used to create the
- * {@link org.apache.arrow.vector.types.pojo.FieldType} of the corresponding {@link
- * org.apache.arrow.vector.FieldVector}.
- *
- *
If there are any {@link java.sql.Types#ARRAY} fields in the {@link java.sql.ResultSet}, the
- * corresponding {@link JdbcFieldInfo} for the array's contents must be defined here. Unfortunately,
- * the sub-type information cannot be retrieved from all JDBC implementations (H2 for example,
- * returns {@link java.sql.Types#NULL} for the array sub-type), so it must be configured here. The
- * column index or name can be used to map to a {@link JdbcFieldInfo}, and that will be used for the
- * conversion.
- */
-public final class JdbcToArrowConfig {
-
- public static final int DEFAULT_TARGET_BATCH_SIZE = 1024;
- public static final int NO_LIMIT_BATCH_SIZE = -1;
- private final Calendar calendar;
- private final BufferAllocator allocator;
- private final boolean includeMetadata;
- private final boolean reuseVectorSchemaRoot;
- private final Map arraySubTypesByColumnIndex;
- private final Map arraySubTypesByColumnName;
- private final Map explicitTypesByColumnIndex;
- private final Map explicitTypesByColumnName;
- private final Map schemaMetadata;
- private final Map> columnMetadataByColumnIndex;
- private final RoundingMode bigDecimalRoundingMode;
- /**
- * The maximum rowCount to read each time when partially convert data. Default value is 1024 and
- * -1 means disable partial read. default is -1 which means disable partial read. Note that this
- * flag only useful for {@link JdbcToArrow#sqlToArrowVectorIterator} 1) if targetBatchSize != -1,
- * it will convert full data into multiple vectors with valueCount no more than targetBatchSize.
- * 2) if targetBatchSize == -1, it will convert full data into a single vector in {@link
- * ArrowVectorIterator}
- */
- private final int targetBatchSize;
-
- private final Function jdbcToArrowTypeConverter;
- private final JdbcConsumerFactory jdbcConsumerGetter;
-
- /**
- * Constructs a new configuration from the provided allocator and calendar. The allocator
- *
is used when constructing the Arrow vectors from the ResultSet, and the calendar is
- * used to define Arrow Timestamp fields, and to read time-based fields from the JDBC
- * ResultSet
.
- *
- * @param allocator The memory allocator to construct the Arrow vectors with.
- * @param calendar The calendar to use when constructing Timestamp fields and reading time-based
- * results.
- */
- JdbcToArrowConfig(BufferAllocator allocator, Calendar calendar) {
- this(
- allocator,
- calendar,
- /* include metadata */ false,
- /* reuse vector schema root */ false,
- /* array sub-types by column index */ null,
- /* array sub-types by column name */ null,
- DEFAULT_TARGET_BATCH_SIZE,
- null,
- null);
- }
-
- JdbcToArrowConfig(
- BufferAllocator allocator,
- Calendar calendar,
- boolean includeMetadata,
- boolean reuseVectorSchemaRoot,
- Map arraySubTypesByColumnIndex,
- Map arraySubTypesByColumnName,
- int targetBatchSize,
- Function jdbcToArrowTypeConverter) {
- this(
- allocator,
- calendar,
- includeMetadata,
- reuseVectorSchemaRoot,
- arraySubTypesByColumnIndex,
- arraySubTypesByColumnName,
- targetBatchSize,
- jdbcToArrowTypeConverter,
- null);
- }
-
- /**
- * Constructs a new configuration from the provided allocator and calendar. The allocator
- *
is used when constructing the Arrow vectors from the ResultSet, and the calendar is
- * used to define Arrow Timestamp fields, and to read time-based fields from the JDBC
- * ResultSet
.
- *
- * @param allocator The memory allocator to construct the Arrow vectors with.
- * @param calendar The calendar to use when constructing Timestamp fields and reading time-based
- * results.
- * @param includeMetadata Whether to include JDBC field metadata in the Arrow Schema Field
- * metadata.
- * @param reuseVectorSchemaRoot Whether to reuse the vector schema root for each data load.
- * @param arraySubTypesByColumnIndex The type of the JDBC array at the column index (1-based).
- * @param arraySubTypesByColumnName The type of the JDBC array at the column name.
- * @param targetBatchSize The target batch size to be used in preallocation of the resulting
- * vectors.
- * @param jdbcToArrowTypeConverter The function that maps JDBC field type information to arrow
- * type. If set to null, the default mapping will be used, which is defined as:
- *
- * - CHAR --> ArrowType.Utf8
- *
- NCHAR --> ArrowType.Utf8
- *
- VARCHAR --> ArrowType.Utf8
- *
- NVARCHAR --> ArrowType.Utf8
- *
- LONGVARCHAR --> ArrowType.Utf8
- *
- LONGNVARCHAR --> ArrowType.Utf8
- *
- NUMERIC --> ArrowType.Decimal(precision, scale)
- *
- DECIMAL --> ArrowType.Decimal(precision, scale)
- *
- BIT --> ArrowType.Bool
- *
- TINYINT --> ArrowType.Int(8, signed)
- *
- SMALLINT --> ArrowType.Int(16, signed)
- *
- INTEGER --> ArrowType.Int(32, signed)
- *
- BIGINT --> ArrowType.Int(64, signed)
- *
- REAL --> ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
- *
- FLOAT --> ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
- *
- DOUBLE --> ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
- *
- BINARY --> ArrowType.Binary
- *
- VARBINARY --> ArrowType.Binary
- *
- LONGVARBINARY --> ArrowType.Binary
- *
- DATE --> ArrowType.Date(DateUnit.DAY)
- *
- TIME --> ArrowType.Time(TimeUnit.MILLISECOND, 32)
- *
- TIMESTAMP --> ArrowType.Timestamp(TimeUnit.MILLISECOND, calendar timezone)
- *
- CLOB --> ArrowType.Utf8
- *
- BLOB --> ArrowType.Binary
- *
- ARRAY --> ArrowType.List
- *
- STRUCT --> ArrowType.Struct
- *
- NULL --> ArrowType.Null
- *
- *
- * @param bigDecimalRoundingMode The java.math.RoundingMode to be used in coercion of a BigDecimal
- * from a ResultSet having a scale which does not match that of the target vector. Use null
- * (default value) to require strict scale matching.
- */
- JdbcToArrowConfig(
- BufferAllocator allocator,
- Calendar calendar,
- boolean includeMetadata,
- boolean reuseVectorSchemaRoot,
- Map arraySubTypesByColumnIndex,
- Map arraySubTypesByColumnName,
- int targetBatchSize,
- Function jdbcToArrowTypeConverter,
- RoundingMode bigDecimalRoundingMode) {
-
- this(
- allocator,
- calendar,
- includeMetadata,
- reuseVectorSchemaRoot,
- arraySubTypesByColumnIndex,
- arraySubTypesByColumnName,
- targetBatchSize,
- jdbcToArrowTypeConverter,
- null,
- null,
- null,
- null,
- bigDecimalRoundingMode);
- }
-
- JdbcToArrowConfig(
- BufferAllocator allocator,
- Calendar calendar,
- boolean includeMetadata,
- boolean reuseVectorSchemaRoot,
- Map arraySubTypesByColumnIndex,
- Map arraySubTypesByColumnName,
- int targetBatchSize,
- Function jdbcToArrowTypeConverter,
- Map explicitTypesByColumnIndex,
- Map explicitTypesByColumnName,
- Map schemaMetadata,
- Map> columnMetadataByColumnIndex,
- RoundingMode bigDecimalRoundingMode) {
- this(
- allocator,
- calendar,
- includeMetadata,
- reuseVectorSchemaRoot,
- arraySubTypesByColumnIndex,
- arraySubTypesByColumnName,
- targetBatchSize,
- jdbcToArrowTypeConverter,
- null,
- explicitTypesByColumnIndex,
- explicitTypesByColumnName,
- schemaMetadata,
- columnMetadataByColumnIndex,
- bigDecimalRoundingMode);
- }
-
- JdbcToArrowConfig(
- BufferAllocator allocator,
- Calendar calendar,
- boolean includeMetadata,
- boolean reuseVectorSchemaRoot,
- Map arraySubTypesByColumnIndex,
- Map arraySubTypesByColumnName,
- int targetBatchSize,
- Function jdbcToArrowTypeConverter,
- JdbcConsumerFactory jdbcConsumerGetter,
- Map explicitTypesByColumnIndex,
- Map explicitTypesByColumnName,
- Map schemaMetadata,
- Map> columnMetadataByColumnIndex,
- RoundingMode bigDecimalRoundingMode) {
- Preconditions.checkNotNull(allocator, "Memory allocator cannot be null");
- this.allocator = allocator;
- this.calendar = calendar;
- this.includeMetadata = includeMetadata;
- this.reuseVectorSchemaRoot = reuseVectorSchemaRoot;
- this.arraySubTypesByColumnIndex = arraySubTypesByColumnIndex;
- this.arraySubTypesByColumnName = arraySubTypesByColumnName;
- this.targetBatchSize = targetBatchSize;
- this.explicitTypesByColumnIndex = explicitTypesByColumnIndex;
- this.explicitTypesByColumnName = explicitTypesByColumnName;
- this.schemaMetadata = schemaMetadata;
- this.columnMetadataByColumnIndex = columnMetadataByColumnIndex;
- this.bigDecimalRoundingMode = bigDecimalRoundingMode;
-
- // set up type converter
- this.jdbcToArrowTypeConverter =
- jdbcToArrowTypeConverter != null
- ? jdbcToArrowTypeConverter
- : (jdbcFieldInfo) -> JdbcToArrowUtils.getArrowTypeFromJdbcType(jdbcFieldInfo, calendar);
-
- this.jdbcConsumerGetter =
- jdbcConsumerGetter != null ? jdbcConsumerGetter : JdbcToArrowUtils::getConsumer;
- }
-
- /**
- * The calendar to use when defining Arrow Timestamp fields and retrieving {@link java.sql.Date},
- * {@link java.sql.Time}, or {@link java.sql.Timestamp} data types from the {@link
- * java.sql.ResultSet}, or null
if not converting.
- *
- * @return the calendar.
- */
- public Calendar getCalendar() {
- return calendar;
- }
-
- /**
- * The Arrow memory allocator.
- *
- * @return the allocator.
- */
- public BufferAllocator getAllocator() {
- return allocator;
- }
-
- /**
- * Whether to include JDBC ResultSet field metadata in the Arrow Schema field metadata.
- *
- * @return true
to include field metadata, false
to exclude it.
- */
- public boolean shouldIncludeMetadata() {
- return includeMetadata;
- }
-
- /** Get the target batch size for partial read. */
- public int getTargetBatchSize() {
- return targetBatchSize;
- }
-
- /** Get whether it is allowed to reuse the vector schema root. */
- public boolean isReuseVectorSchemaRoot() {
- return reuseVectorSchemaRoot;
- }
-
- /** Gets the mapping between JDBC type information to Arrow type. */
- public Function getJdbcToArrowTypeConverter() {
- return jdbcToArrowTypeConverter;
- }
-
- /** Gets the JDBC consumer getter. */
- public JdbcConsumerFactory getJdbcConsumerGetter() {
- return jdbcConsumerGetter;
- }
-
- /**
- * Returns the array sub-type {@link JdbcFieldInfo} defined for the provided column index.
- *
- * @param index The {@link java.sql.ResultSetMetaData} column index of an {@link
- * java.sql.Types#ARRAY} type.
- * @return The {@link JdbcFieldInfo} for that array's sub-type, or null
if not
- * defined.
- */
- public JdbcFieldInfo getArraySubTypeByColumnIndex(int index) {
- if (arraySubTypesByColumnIndex == null) {
- return null;
- } else {
- return arraySubTypesByColumnIndex.get(index);
- }
- }
-
- /**
- * Returns the array sub-type {@link JdbcFieldInfo} defined for the provided column name.
- *
- * @param name The {@link java.sql.ResultSetMetaData} column name of an {@link
- * java.sql.Types#ARRAY} type.
- * @return The {@link JdbcFieldInfo} for that array's sub-type, or null
if not
- * defined.
- */
- public JdbcFieldInfo getArraySubTypeByColumnName(String name) {
- if (arraySubTypesByColumnName == null) {
- return null;
- } else {
- return arraySubTypesByColumnName.get(name);
- }
- }
-
- /**
- * Returns the type {@link JdbcFieldInfo} explicitly defined for the provided column index.
- *
- * @param index The {@link java.sql.ResultSetMetaData} column index to evaluate for explicit type
- * mapping.
- * @return The {@link JdbcFieldInfo} defined for the column, or null
if not defined.
- */
- public JdbcFieldInfo getExplicitTypeByColumnIndex(int index) {
- if (explicitTypesByColumnIndex == null) {
- return null;
- } else {
- return explicitTypesByColumnIndex.get(index);
- }
- }
-
- /**
- * Returns the type {@link JdbcFieldInfo} explicitly defined for the provided column name.
- *
- * @param name The {@link java.sql.ResultSetMetaData} column name to evaluate for explicit type
- * mapping.
- * @return The {@link JdbcFieldInfo} defined for the column, or null
if not defined.
- */
- public JdbcFieldInfo getExplicitTypeByColumnName(String name) {
- if (explicitTypesByColumnName == null) {
- return null;
- } else {
- return explicitTypesByColumnName.get(name);
- }
- }
-
- /** Return schema level metadata or null if not provided. */
- public Map getSchemaMetadata() {
- return schemaMetadata;
- }
-
- /** Return metadata from columnIndex->meta map on per field basis or null if not provided. */
- public Map> getColumnMetadataByColumnIndex() {
- return columnMetadataByColumnIndex;
- }
-
- public RoundingMode getBigDecimalRoundingMode() {
- return bigDecimalRoundingMode;
- }
-
- /** Interface for a function that gets a JDBC consumer for the given values. */
- @FunctionalInterface
- public interface JdbcConsumerFactory {
- JdbcConsumer apply(
- ArrowType arrowType,
- int columnIndex,
- boolean nullable,
- FieldVector vector,
- JdbcToArrowConfig config);
- }
-}
diff --git a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfigBuilder.java b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfigBuilder.java
deleted file mode 100644
index ea9ffe55d334a..0000000000000
--- a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfigBuilder.java
+++ /dev/null
@@ -1,292 +0,0 @@
-/*
- * 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 static org.apache.arrow.adapter.jdbc.JdbcToArrowConfig.DEFAULT_TARGET_BATCH_SIZE;
-
-import java.math.RoundingMode;
-import java.util.Calendar;
-import java.util.Map;
-import java.util.function.Function;
-import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.util.Preconditions;
-import org.apache.arrow.vector.FieldVector;
-import org.apache.arrow.vector.types.pojo.ArrowType;
-
-/** This class builds {@link JdbcToArrowConfig}s. */
-public class JdbcToArrowConfigBuilder {
- private Calendar calendar;
- private BufferAllocator allocator;
- private boolean includeMetadata;
- private boolean reuseVectorSchemaRoot;
- private Map arraySubTypesByColumnIndex;
- private Map arraySubTypesByColumnName;
- private Map explicitTypesByColumnIndex;
- private Map explicitTypesByColumnName;
- private Map schemaMetadata;
- private Map> columnMetadataByColumnIndex;
- private int targetBatchSize;
- private Function jdbcToArrowTypeConverter;
- private JdbcToArrowConfig.JdbcConsumerFactory jdbcConsumerGetter;
- private RoundingMode bigDecimalRoundingMode;
-
- /**
- * Default constructor for the JdbcToArrowConfigBuilder}
. Use the setter methods for
- * the allocator and calendar; the allocator must be set. Otherwise, {@link #build()} will throw a
- * {@link NullPointerException}.
- */
- public JdbcToArrowConfigBuilder() {
- this.allocator = null;
- this.calendar = null;
- this.includeMetadata = false;
- this.reuseVectorSchemaRoot = false;
- this.arraySubTypesByColumnIndex = null;
- this.arraySubTypesByColumnName = null;
- this.explicitTypesByColumnIndex = null;
- this.explicitTypesByColumnName = null;
- this.schemaMetadata = null;
- this.columnMetadataByColumnIndex = null;
- this.bigDecimalRoundingMode = null;
- }
-
- /**
- * Constructor for the JdbcToArrowConfigBuilder
. The allocator is required, and a
- * {@link NullPointerException} will be thrown if it is null
.
- *
- * The allocator is used to construct Arrow vectors from the JDBC ResultSet. The calendar is
- * used to determine the time zone of {@link java.sql.Timestamp} fields and convert {@link
- * java.sql.Date}, {@link java.sql.Time}, and {@link java.sql.Timestamp} fields to a single,
- * common time zone when reading from the result set.
- *
- * @param allocator The Arrow Vector memory allocator.
- * @param calendar The calendar to use when constructing timestamp fields.
- */
- public JdbcToArrowConfigBuilder(BufferAllocator allocator, Calendar calendar) {
- this();
-
- Preconditions.checkNotNull(allocator, "Memory allocator cannot be null");
-
- this.allocator = allocator;
- this.calendar = calendar;
- this.includeMetadata = false;
- this.reuseVectorSchemaRoot = false;
- this.targetBatchSize = DEFAULT_TARGET_BATCH_SIZE;
- }
-
- /**
- * Constructor for the JdbcToArrowConfigBuilder
. Both the allocator and calendar are
- * required. A {@link NullPointerException} will be thrown if either of those arguments is
- * null
.
- *
- *
The allocator is used to construct Arrow vectors from the JDBC ResultSet. The calendar is
- * used to determine the time zone of {@link java.sql.Timestamp} fields and convert {@link
- * java.sql.Date}, {@link java.sql.Time}, and {@link java.sql.Timestamp} fields to a single,
- * common time zone when reading from the result set.
- *
- *
The includeMetadata
argument, if true
will cause various
- * information about each database field to be added to the Vector Schema's field metadata.
- *
- * @param allocator The Arrow Vector memory allocator.
- * @param calendar The calendar to use when constructing timestamp fields.
- */
- public JdbcToArrowConfigBuilder(
- BufferAllocator allocator, Calendar calendar, boolean includeMetadata) {
- this(allocator, calendar);
- this.includeMetadata = includeMetadata;
- }
-
- /**
- * Sets the memory allocator to use when constructing the Arrow vectors from the ResultSet.
- *
- * @param allocator the allocator to set.
- * @exception NullPointerException if allocator
is null.
- */
- public JdbcToArrowConfigBuilder setAllocator(BufferAllocator allocator) {
- Preconditions.checkNotNull(allocator, "Memory allocator cannot be null");
- this.allocator = allocator;
- return this;
- }
-
- /**
- * Sets the {@link Calendar} to use when constructing timestamp fields in the Arrow schema, and
- * reading time-based fields from the JDBC ResultSet
.
- *
- * @param calendar the calendar to set.
- */
- public JdbcToArrowConfigBuilder setCalendar(Calendar calendar) {
- this.calendar = calendar;
- return this;
- }
-
- /**
- * Sets whether to include JDBC ResultSet field metadata in the Arrow Schema field metadata.
- *
- * @param includeMetadata Whether to include or exclude JDBC metadata in the Arrow Schema field
- * metadata.
- * @return This instance of the JdbcToArrowConfig
, for chaining.
- */
- public JdbcToArrowConfigBuilder setIncludeMetadata(boolean includeMetadata) {
- this.includeMetadata = includeMetadata;
- return this;
- }
-
- /**
- * Sets the mapping of column-index-to-{@link JdbcFieldInfo} used for columns of type {@link
- * java.sql.Types#ARRAY}. The column index is 1-based, to match the JDBC column index.
- *
- * @param map The mapping.
- * @return This instance of the JdbcToArrowConfig
, for chaining.
- */
- public JdbcToArrowConfigBuilder setArraySubTypeByColumnIndexMap(Map map) {
- this.arraySubTypesByColumnIndex = map;
- return this;
- }
-
- /**
- * Sets the mapping of column-name-to-{@link JdbcFieldInfo} used for columns of type {@link
- * java.sql.Types#ARRAY}.
- *
- * @param map The mapping.
- * @return This instance of the JdbcToArrowConfig
, for chaining.
- */
- public JdbcToArrowConfigBuilder setArraySubTypeByColumnNameMap(Map map) {
- this.arraySubTypesByColumnName = map;
- return this;
- }
-
- /**
- * Sets the mapping of column-index-to-{@link JdbcFieldInfo} used for column types.
- *
- * This can be useful to override type information from JDBC drivers that provide incomplete
- * type info, e.g. DECIMAL with precision = scale = 0.
- *
- *
The column index is 1-based, to match the JDBC column index.
- *
- * @param map The mapping.
- */
- public JdbcToArrowConfigBuilder setExplicitTypesByColumnIndex(Map map) {
- this.explicitTypesByColumnIndex = map;
- return this;
- }
-
- /**
- * Sets the mapping of column-name-to-{@link JdbcFieldInfo} used for column types.
- *
- * This can be useful to override type information from JDBC drivers that provide incomplete
- * type info, e.g. DECIMAL with precision = scale = 0.
- *
- * @param map The mapping.
- */
- public JdbcToArrowConfigBuilder setExplicitTypesByColumnName(Map map) {
- this.explicitTypesByColumnName = map;
- return this;
- }
-
- /**
- * Set the target number of rows to convert at once.
- *
- * Use {@link JdbcToArrowConfig#NO_LIMIT_BATCH_SIZE} to read all rows at once.
- */
- public JdbcToArrowConfigBuilder setTargetBatchSize(int targetBatchSize) {
- this.targetBatchSize = targetBatchSize;
- return this;
- }
-
- /**
- * Set the function used to convert JDBC types to Arrow types.
- *
- *
Defaults to wrapping {@link JdbcToArrowUtils#getArrowTypeFromJdbcType(JdbcFieldInfo,
- * Calendar)}.
- *
- * @see JdbcToArrowUtils#reportUnsupportedTypesAsUnknown(Function)
- */
- public JdbcToArrowConfigBuilder setJdbcToArrowTypeConverter(
- Function jdbcToArrowTypeConverter) {
- this.jdbcToArrowTypeConverter = jdbcToArrowTypeConverter;
- return this;
- }
-
- /**
- * Set the function used to get a JDBC consumer for a given type.
- *
- * Defaults to wrapping {@link JdbcToArrowUtils#getConsumer(ArrowType, Integer, Boolean,
- * FieldVector, JdbcToArrowConfig)}.
- */
- public JdbcToArrowConfigBuilder setJdbcConsumerGetter(
- JdbcToArrowConfig.JdbcConsumerFactory jdbcConsumerGetter) {
- this.jdbcConsumerGetter = jdbcConsumerGetter;
- return this;
- }
-
- /**
- * Set whether to use the same {@link org.apache.arrow.vector.VectorSchemaRoot} instance on each
- * iteration, or to allocate a new one.
- */
- public JdbcToArrowConfigBuilder setReuseVectorSchemaRoot(boolean reuseVectorSchemaRoot) {
- this.reuseVectorSchemaRoot = reuseVectorSchemaRoot;
- return this;
- }
-
- /** Set metadata for schema. */
- public JdbcToArrowConfigBuilder setSchemaMetadata(Map schemaMetadata) {
- this.schemaMetadata = schemaMetadata;
- return this;
- }
-
- /** Set metadata from columnIndex->meta map on per field basis. */
- public JdbcToArrowConfigBuilder setColumnMetadataByColumnIndex(
- Map> columnMetadataByColumnIndex) {
- this.columnMetadataByColumnIndex = columnMetadataByColumnIndex;
- return this;
- }
-
- /**
- * Set the rounding mode used when the scale of the actual value does not match the declared
- * scale.
- *
- * By default, an error is raised in such cases.
- */
- public JdbcToArrowConfigBuilder setBigDecimalRoundingMode(RoundingMode bigDecimalRoundingMode) {
- this.bigDecimalRoundingMode = bigDecimalRoundingMode;
- return this;
- }
-
- /**
- * This builds the {@link JdbcToArrowConfig} from the provided {@link BufferAllocator} and {@link
- * Calendar}.
- *
- * @return The built {@link JdbcToArrowConfig}
- * @throws NullPointerException if either the allocator or calendar was not set.
- */
- public JdbcToArrowConfig build() {
- return new JdbcToArrowConfig(
- allocator,
- calendar,
- includeMetadata,
- reuseVectorSchemaRoot,
- arraySubTypesByColumnIndex,
- arraySubTypesByColumnName,
- targetBatchSize,
- jdbcToArrowTypeConverter,
- jdbcConsumerGetter,
- explicitTypesByColumnIndex,
- explicitTypesByColumnName,
- schemaMetadata,
- columnMetadataByColumnIndex,
- bigDecimalRoundingMode);
- }
-}
diff --git a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowUtils.java b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowUtils.java
deleted file mode 100644
index aecb734a8bbf7..0000000000000
--- a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowUtils.java
+++ /dev/null
@@ -1,567 +0,0 @@
-/*
- * 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 static org.apache.arrow.vector.types.FloatingPointPrecision.DOUBLE;
-import static org.apache.arrow.vector.types.FloatingPointPrecision.SINGLE;
-import static org.apache.arrow.vector.types.Types.MinorType;
-
-import java.io.IOException;
-import java.math.RoundingMode;
-import java.sql.Date;
-import java.sql.ParameterMetaData;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.sql.Types;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Calendar;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.TimeZone;
-import java.util.function.Function;
-import org.apache.arrow.adapter.jdbc.consumer.ArrayConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.BigIntConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.BinaryConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.BitConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.CompositeJdbcConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.DateConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.Decimal256Consumer;
-import org.apache.arrow.adapter.jdbc.consumer.DecimalConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.DoubleConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.FloatConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.IntConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.JdbcConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.MapConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.NullConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.SmallIntConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.TimeConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.TimestampConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.TimestampTZConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.TinyIntConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.VarCharConsumer;
-import org.apache.arrow.adapter.jdbc.consumer.exceptions.JdbcConsumerException;
-import org.apache.arrow.memory.RootAllocator;
-import org.apache.arrow.util.Preconditions;
-import org.apache.arrow.vector.BigIntVector;
-import org.apache.arrow.vector.BitVector;
-import org.apache.arrow.vector.DateDayVector;
-import org.apache.arrow.vector.Decimal256Vector;
-import org.apache.arrow.vector.DecimalVector;
-import org.apache.arrow.vector.FieldVector;
-import org.apache.arrow.vector.Float4Vector;
-import org.apache.arrow.vector.Float8Vector;
-import org.apache.arrow.vector.IntVector;
-import org.apache.arrow.vector.NullVector;
-import org.apache.arrow.vector.SmallIntVector;
-import org.apache.arrow.vector.TimeMilliVector;
-import org.apache.arrow.vector.TimeStampMilliTZVector;
-import org.apache.arrow.vector.TimeStampMilliVector;
-import org.apache.arrow.vector.TinyIntVector;
-import org.apache.arrow.vector.VarBinaryVector;
-import org.apache.arrow.vector.VarCharVector;
-import org.apache.arrow.vector.VectorSchemaRoot;
-import org.apache.arrow.vector.complex.ListVector;
-import org.apache.arrow.vector.complex.MapVector;
-import org.apache.arrow.vector.extension.OpaqueType;
-import org.apache.arrow.vector.types.DateUnit;
-import org.apache.arrow.vector.types.TimeUnit;
-import org.apache.arrow.vector.types.pojo.ArrowType;
-import org.apache.arrow.vector.types.pojo.Field;
-import org.apache.arrow.vector.types.pojo.FieldType;
-import org.apache.arrow.vector.types.pojo.Schema;
-import org.apache.arrow.vector.util.ValueVectorUtility;
-
-/**
- * Class that does most of the work to convert JDBC ResultSet data into Arrow columnar format Vector
- * objects.
- *
- * @since 0.10.0
- */
-public class JdbcToArrowUtils {
-
- private static final int JDBC_ARRAY_VALUE_COLUMN = 2;
-
- /** Returns the instance of a {java.util.Calendar} with the UTC time zone and root locale. */
- public static Calendar getUtcCalendar() {
- return Calendar.getInstance(TimeZone.getTimeZone("UTC"), Locale.ROOT);
- }
-
- /**
- * Create Arrow {@link Schema} object for the given JDBC {@link ResultSetMetaData}.
- *
- * @param rsmd The ResultSetMetaData containing the results, to read the JDBC metadata from.
- * @param calendar The calendar to use the time zone field of, to construct Timestamp fields from.
- * @return {@link Schema}
- * @throws SQLException on error
- */
- public static Schema jdbcToArrowSchema(ResultSetMetaData rsmd, Calendar calendar)
- throws SQLException {
- Preconditions.checkNotNull(calendar, "Calendar object can't be null");
-
- return jdbcToArrowSchema(rsmd, new JdbcToArrowConfig(new RootAllocator(0), calendar));
- }
-
- /**
- * Create Arrow {@link Schema} object for the given JDBC {@link ResultSetMetaData}.
- *
- * @param parameterMetaData The ResultSetMetaData containing the results, to read the JDBC
- * metadata from.
- * @param calendar The calendar to use the time zone field of, to construct Timestamp fields from.
- * @return {@link Schema}
- * @throws SQLException on error
- */
- public static Schema jdbcToArrowSchema(
- final ParameterMetaData parameterMetaData, final Calendar calendar) throws SQLException {
- Preconditions.checkNotNull(calendar, "Calendar object can't be null");
- Preconditions.checkNotNull(parameterMetaData);
- final List parameterFields = new ArrayList<>(parameterMetaData.getParameterCount());
- for (int parameterCounter = 1;
- parameterCounter <= parameterMetaData.getParameterCount();
- parameterCounter++) {
- final int jdbcDataType = parameterMetaData.getParameterType(parameterCounter);
- final int jdbcIsNullable = parameterMetaData.isNullable(parameterCounter);
- final boolean arrowIsNullable = jdbcIsNullable != ParameterMetaData.parameterNoNulls;
- final int precision = parameterMetaData.getPrecision(parameterCounter);
- final int scale = parameterMetaData.getScale(parameterCounter);
- final ArrowType arrowType =
- getArrowTypeFromJdbcType(new JdbcFieldInfo(jdbcDataType, precision, scale), calendar);
- final FieldType fieldType = new FieldType(arrowIsNullable, arrowType, /*dictionary=*/ null);
- parameterFields.add(new Field(null, fieldType, null));
- }
-
- return new Schema(parameterFields);
- }
-
- /**
- * Converts the provided JDBC type to its respective {@link ArrowType} counterpart.
- *
- * @param fieldInfo the {@link JdbcFieldInfo} with information about the original JDBC type.
- * @param calendar the {@link Calendar} to use for datetime data types.
- * @return a new {@link ArrowType}.
- */
- public static ArrowType getArrowTypeFromJdbcType(
- final JdbcFieldInfo fieldInfo, final Calendar calendar) {
- switch (fieldInfo.getJdbcType()) {
- case Types.BOOLEAN:
- case Types.BIT:
- return new ArrowType.Bool();
- case Types.TINYINT:
- return new ArrowType.Int(8, true);
- case Types.SMALLINT:
- return new ArrowType.Int(16, true);
- case Types.INTEGER:
- return new ArrowType.Int(32, true);
- case Types.BIGINT:
- return new ArrowType.Int(64, true);
- case Types.NUMERIC:
- case Types.DECIMAL:
- int precision = fieldInfo.getPrecision();
- int scale = fieldInfo.getScale();
- if (precision > 38) {
- return new ArrowType.Decimal(precision, scale, 256);
- } else {
- return new ArrowType.Decimal(precision, scale, 128);
- }
- case Types.REAL:
- case Types.FLOAT:
- return new ArrowType.FloatingPoint(SINGLE);
- case Types.DOUBLE:
- return new ArrowType.FloatingPoint(DOUBLE);
- case Types.CHAR:
- case Types.NCHAR:
- case Types.VARCHAR:
- case Types.NVARCHAR:
- case Types.LONGVARCHAR:
- case Types.LONGNVARCHAR:
- case Types.CLOB:
- return new ArrowType.Utf8();
- case Types.DATE:
- return new ArrowType.Date(DateUnit.DAY);
- case Types.TIME:
- return new ArrowType.Time(TimeUnit.MILLISECOND, 32);
- case Types.TIMESTAMP:
- final String timezone;
- if (calendar != null) {
- timezone = calendar.getTimeZone().getID();
- } else {
- timezone = null;
- }
- return new ArrowType.Timestamp(TimeUnit.MILLISECOND, timezone);
- case Types.BINARY:
- case Types.VARBINARY:
- case Types.LONGVARBINARY:
- case Types.BLOB:
- return new ArrowType.Binary();
- case Types.ARRAY:
- return new ArrowType.List();
- case Types.NULL:
- return new ArrowType.Null();
- case Types.STRUCT:
- return new ArrowType.Struct();
- default:
- throw new UnsupportedOperationException("Unmapped JDBC type: " + fieldInfo.getJdbcType());
- }
- }
-
- /**
- * Wrap a JDBC to Arrow type converter such that {@link UnsupportedOperationException} becomes
- * {@link OpaqueType}.
- *
- * @param typeConverter The type converter to wrap.
- * @param vendorName The database name to report as the Opaque type's vendor name.
- */
- public static Function