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/01/26 18:12:17 UTC

[GitHub] [arrow] lidavidm commented on a change in pull request #12254: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

lidavidm commented on a change in pull request #12254:
URL: https://github.com/apache/arrow/pull/12254#discussion_r792699352



##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/accessor/ArrowFlightJdbcAccessorFactory.java
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.driver.jdbc.accessor;
+
+import java.util.function.IntSupplier;
+
+import org.apache.arrow.driver.jdbc.accessor.impl.ArrowFlightJdbcNullVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.binary.ArrowFlightJdbcBinaryVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcDateVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcDurationVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcIntervalVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeStampVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcDenseUnionVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcFixedSizeListVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcLargeListVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcListVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcMapVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcStructVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcUnionVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.numeric.ArrowFlightJdbcBaseIntVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.numeric.ArrowFlightJdbcBitVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.numeric.ArrowFlightJdbcDecimalVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.numeric.ArrowFlightJdbcFloat4VectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.numeric.ArrowFlightJdbcFloat8VectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.text.ArrowFlightJdbcVarCharVectorAccessor;
+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.DurationVector;
+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.IntervalDayVector;
+import org.apache.arrow.vector.IntervalYearVector;
+import org.apache.arrow.vector.LargeVarBinaryVector;
+import org.apache.arrow.vector.LargeVarCharVector;
+import org.apache.arrow.vector.NullVector;
+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.UInt1Vector;
+import org.apache.arrow.vector.UInt2Vector;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.arrow.vector.UInt8Vector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.complex.DenseUnionVector;
+import org.apache.arrow.vector.complex.FixedSizeListVector;
+import org.apache.arrow.vector.complex.LargeListVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.MapVector;
+import org.apache.arrow.vector.complex.StructVector;
+import org.apache.arrow.vector.complex.UnionVector;
+
+/**
+ * Factory to instantiate the accessors.
+ */
+public class ArrowFlightJdbcAccessorFactory {
+
+  /**
+   * Create an accessor according to the its type.
+   *
+   * @param vector        an instance of an arrow vector.
+   * @param getCurrentRow a supplier to check which row is being accessed.
+   * @return an instance of one of the accessors.
+   */
+  public static ArrowFlightJdbcAccessor createAccessor(ValueVector vector,
+                                                       IntSupplier getCurrentRow,
+                                                       WasNullConsumer setCursorWasNull) {
+    if (vector instanceof UInt1Vector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((UInt1Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof UInt2Vector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((UInt2Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof UInt4Vector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((UInt4Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof UInt8Vector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((UInt8Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof TinyIntVector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((TinyIntVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof SmallIntVector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((SmallIntVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof IntVector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((IntVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof BigIntVector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((BigIntVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof Float4Vector) {
+      return new ArrowFlightJdbcFloat4VectorAccessor((Float4Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof Float8Vector) {
+      return new ArrowFlightJdbcFloat8VectorAccessor((Float8Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof BitVector) {
+      return new ArrowFlightJdbcBitVectorAccessor((BitVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof DecimalVector) {
+      return new ArrowFlightJdbcDecimalVectorAccessor((DecimalVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof Decimal256Vector) {
+      return new ArrowFlightJdbcDecimalVectorAccessor((Decimal256Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof VarBinaryVector) {
+      return new ArrowFlightJdbcBinaryVectorAccessor((VarBinaryVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof LargeVarBinaryVector) {
+      return new ArrowFlightJdbcBinaryVectorAccessor((LargeVarBinaryVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof FixedSizeBinaryVector) {
+      return new ArrowFlightJdbcBinaryVectorAccessor((FixedSizeBinaryVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof TimeStampVector) {
+      return new ArrowFlightJdbcTimeStampVectorAccessor((TimeStampVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof TimeNanoVector) {
+      return new ArrowFlightJdbcTimeVectorAccessor((TimeNanoVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof TimeMicroVector) {
+      return new ArrowFlightJdbcTimeVectorAccessor((TimeMicroVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof TimeMilliVector) {
+      return new ArrowFlightJdbcTimeVectorAccessor((TimeMilliVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof TimeSecVector) {
+      return new ArrowFlightJdbcTimeVectorAccessor((TimeSecVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof DateDayVector) {
+      return new ArrowFlightJdbcDateVectorAccessor(((DateDayVector) vector), getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof DateMilliVector) {
+      return new ArrowFlightJdbcDateVectorAccessor(((DateMilliVector) vector), getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof VarCharVector) {
+      return new ArrowFlightJdbcVarCharVectorAccessor((VarCharVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof LargeVarCharVector) {
+      return new ArrowFlightJdbcVarCharVectorAccessor((LargeVarCharVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof DurationVector) {
+      return new ArrowFlightJdbcDurationVectorAccessor((DurationVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof IntervalDayVector) {
+      return new ArrowFlightJdbcIntervalVectorAccessor(((IntervalDayVector) vector), getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof IntervalYearVector) {
+      return new ArrowFlightJdbcIntervalVectorAccessor(((IntervalYearVector) vector), getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof StructVector) {
+      return new ArrowFlightJdbcStructVectorAccessor((StructVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof MapVector) {
+      return new ArrowFlightJdbcMapVectorAccessor((MapVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof ListVector) {
+      return new ArrowFlightJdbcListVectorAccessor((ListVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof LargeListVector) {
+      return new ArrowFlightJdbcLargeListVectorAccessor((LargeListVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof FixedSizeListVector) {
+      return new ArrowFlightJdbcFixedSizeListVectorAccessor((FixedSizeListVector) vector,
+          getCurrentRow, setCursorWasNull);
+    } else if (vector instanceof UnionVector) {
+      return new ArrowFlightJdbcUnionVectorAccessor((UnionVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof DenseUnionVector) {
+      return new ArrowFlightJdbcDenseUnionVectorAccessor((DenseUnionVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof NullVector || vector == null) {
+      return new ArrowFlightJdbcNullVectorAccessor(setCursorWasNull);
+    }
+
+    throw new UnsupportedOperationException();

Review comment:
       Maybe include the type in the exception message?

##########
File path: java/pom.xml
##########
@@ -506,12 +506,23 @@
             </lifecycleMappingMetadata>
           </configuration>
         </plugin>
+        <plugin>
+          <groupId>com.github.spotbugs</groupId>
+          <artifactId>spotbugs-maven-plugin</artifactId>
+          <version>4.2.3</version>
+        </plugin>
+

Review comment:
       Is it possible to keep these plugins/dependencies in the driver's pom.xml? (Though it would be nice to enable static analyzers project-wide, that should be done separately.)

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/accessor/impl/calendar/ArrowFlightJdbcTimeStampVectorAccessor.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.driver.jdbc.accessor.impl.calendar;
+
+import static org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeStampVectorGetter.Getter;
+import static org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeStampVectorGetter.Holder;
+import static org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeStampVectorGetter.createGetter;
+
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDateTime;
+import java.time.temporal.ChronoUnit;
+import java.util.Calendar;
+import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
+import java.util.function.IntSupplier;
+
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessor;
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessorFactory;
+import org.apache.arrow.vector.TimeStampVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.util.DateUtility;
+
+/**
+ * Accessor for the Arrow types extending from {@link TimeStampVector}.
+ */
+public class ArrowFlightJdbcTimeStampVectorAccessor extends ArrowFlightJdbcAccessor {
+
+  private final TimeZone timeZone;
+  private final Getter getter;
+  private final TimeUnit timeUnit;
+  private final LongToLocalDateTime longToLocalDateTime;
+  private final Holder holder;
+
+  /**
+   * Functional interface used to convert a number (in any time resolution) to LocalDateTime.
+   */
+  interface LongToLocalDateTime {
+    LocalDateTime fromLong(long value);
+  }
+
+  /**
+   * Instantiate a ArrowFlightJdbcTimeStampVectorAccessor for given vector.
+   */
+  public ArrowFlightJdbcTimeStampVectorAccessor(TimeStampVector vector,
+                                                IntSupplier currentRowSupplier,
+                                                ArrowFlightJdbcAccessorFactory.WasNullConsumer setCursorWasNull) {
+    super(currentRowSupplier, setCursorWasNull);
+    this.holder = new Holder();
+    this.getter = createGetter(vector);
+
+    this.timeZone = getTimeZoneForVector(vector);
+    this.timeUnit = getTimeUnitForVector(vector);
+    this.longToLocalDateTime = getLongToLocalDateTimeForVector(vector, this.timeZone);
+  }
+
+  @Override
+  public Class<?> getObjectClass() {
+    return Timestamp.class;
+  }
+
+  @Override
+  public Object getObject() {
+    return this.getTimestamp(null);
+  }
+
+  private LocalDateTime getLocalDateTime(Calendar calendar) {
+    getter.get(getCurrentRow(), holder);
+    this.wasNull = holder.isSet == 0;
+    this.wasNullConsumer.setWasNull(this.wasNull);
+    if (this.wasNull) {
+      return null;
+    }
+
+    long value = holder.value;
+
+    LocalDateTime localDateTime = this.longToLocalDateTime.fromLong(value);
+
+    if (calendar != null) {
+      TimeZone timeZone = calendar.getTimeZone();
+      long millis = this.timeUnit.toMillis(value);
+      localDateTime = localDateTime
+          .minus(timeZone.getOffset(millis) - this.timeZone.getOffset(millis), ChronoUnit.MILLIS);
+    }
+    return localDateTime;
+  }
+
+  @Override
+  public Date getDate(Calendar calendar) {
+    LocalDateTime localDateTime = getLocalDateTime(calendar);
+    if (localDateTime == null) {
+      return null;
+    }
+
+    return new Date(Timestamp.valueOf(localDateTime).getTime());
+  }
+
+  @Override
+  public Time getTime(Calendar calendar) {
+    LocalDateTime localDateTime = getLocalDateTime(calendar);
+    if (localDateTime == null) {
+      return null;
+    }
+
+    return new Time(Timestamp.valueOf(localDateTime).getTime());
+  }
+
+  @Override
+  public Timestamp getTimestamp(Calendar calendar) {
+    LocalDateTime localDateTime = getLocalDateTime(calendar);
+    if (localDateTime == null) {
+      return null;
+    }
+
+    return Timestamp.valueOf(localDateTime);
+  }
+
+  protected static TimeUnit getTimeUnitForVector(TimeStampVector vector) {
+    ArrowType.Timestamp arrowType =
+        (ArrowType.Timestamp) vector.getField().getFieldType().getType();
+
+    switch (arrowType.getUnit()) {
+      case NANOSECOND:
+        return TimeUnit.NANOSECONDS;
+      case MICROSECOND:
+        return TimeUnit.MICROSECONDS;
+      case MILLISECOND:
+        return TimeUnit.MILLISECONDS;
+      case SECOND:
+        return TimeUnit.SECONDS;
+      default:
+        throw new UnsupportedOperationException("Invalid Arrow time unit");
+    }
+  }
+
+  protected static LongToLocalDateTime getLongToLocalDateTimeForVector(TimeStampVector vector,
+                                                                       TimeZone timeZone) {
+    String timeZoneID = timeZone.getID();
+
+    ArrowType.Timestamp arrowType =
+        (ArrowType.Timestamp) vector.getField().getFieldType().getType();
+
+    switch (arrowType.getUnit()) {
+      case NANOSECOND:
+        return nanoseconds -> DateUtility.getLocalDateTimeFromEpochNano(nanoseconds, timeZoneID);
+      case MICROSECOND:
+        return microseconds -> DateUtility.getLocalDateTimeFromEpochMicro(microseconds, timeZoneID);
+      case MILLISECOND:
+        return milliseconds -> DateUtility.getLocalDateTimeFromEpochMilli(milliseconds, timeZoneID);
+      case SECOND:
+        return seconds -> DateUtility.getLocalDateTimeFromEpochMilli(
+            TimeUnit.SECONDS.toMillis(seconds), timeZoneID);
+      default:
+        throw new UnsupportedOperationException("Invalid Arrow time unit");
+    }
+  }
+
+  protected static TimeZone getTimeZoneForVector(TimeStampVector vector) {
+    ArrowType.Timestamp arrowType =
+        (ArrowType.Timestamp) vector.getField().getFieldType().getType();
+
+    String timezoneName = arrowType.getTimezone();
+    if (timezoneName == null) {
+      return TimeZone.getDefault();

Review comment:
       Using the _default_ timezone seems wrong, doesn't it? It's machine-dependent. 

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/accessor/impl/numeric/ArrowFlightJdbcFloat4VectorAccessor.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.driver.jdbc.accessor.impl.numeric;
+
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.nio.ByteBuffer;
+import java.util.function.IntSupplier;
+
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessor;
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessorFactory;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.holders.NullableFloat4Holder;
+
+/**
+ * Accessor for the Float4Vector.
+ */
+public class ArrowFlightJdbcFloat4VectorAccessor extends ArrowFlightJdbcAccessor {
+
+  private final Float4Vector vector;
+  private final NullableFloat4Holder holder;
+
+  /**
+   * Instantiate a accessor for the {@link Float4Vector}.
+   *
+   * @param vector             an instance of a Float4Vector.
+   * @param currentRowSupplier the supplier to track the lines.
+   * @param setCursorWasNull   the consumer to set if value was null.
+   */
+  public ArrowFlightJdbcFloat4VectorAccessor(Float4Vector vector,
+                                             IntSupplier currentRowSupplier,
+                                             ArrowFlightJdbcAccessorFactory.WasNullConsumer setCursorWasNull) {
+    super(currentRowSupplier, setCursorWasNull);
+    this.holder = new NullableFloat4Holder();
+    this.vector = vector;
+  }
+
+  @Override
+  public Class<?> getObjectClass() {
+    return Float.class;
+  }
+
+  @Override
+  public String getString() {
+    final float value = this.getFloat();
+
+    return this.wasNull ? null : Float.toString(value);
+  }
+
+  @Override
+  public boolean getBoolean() {
+    return this.getFloat() != 0.0;
+  }
+
+  @Override
+  public byte getByte() {
+    return (byte) this.getFloat();
+  }
+
+  @Override
+  public short getShort() {
+    return (short) this.getFloat();
+  }
+
+  @Override
+  public int getInt() {
+    return (int) this.getFloat();
+  }
+
+  @Override
+  public long getLong() {
+    return (long) this.getFloat();
+  }
+
+  @Override
+  public float getFloat() {
+    vector.get(getCurrentRow(), holder);
+
+    this.wasNull = holder.isSet == 0;
+    this.wasNullConsumer.setWasNull(this.wasNull);
+    if (this.wasNull) {
+      return 0;
+    }
+
+    return holder.value;
+  }
+
+  @Override
+  public double getDouble() {
+    return this.getFloat();
+  }
+
+  @Override
+  public BigDecimal getBigDecimal() {
+    final float value = this.getFloat();
+
+    final boolean infinite = Float.isInfinite(value);
+    if (infinite) {
+      throw new UnsupportedOperationException();

Review comment:
       Can we include a message?

##########
File path: java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlColumnMetadata.java
##########
@@ -0,0 +1,295 @@
+/*

Review comment:
       I'm going to assume any changes under flight-sql are from the other PRs…let me know if that's not the case.

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/accessor/impl/numeric/ArrowFlightJdbcFloat8VectorAccessor.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.driver.jdbc.accessor.impl.numeric;
+
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.nio.ByteBuffer;
+import java.util.function.IntSupplier;
+
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessor;
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessorFactory;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.holders.NullableFloat8Holder;
+
+/**
+ * Accessor for the Float8Vector.
+ */
+public class ArrowFlightJdbcFloat8VectorAccessor extends ArrowFlightJdbcAccessor {
+
+  private final Float8Vector vector;
+  private final NullableFloat8Holder holder;
+
+  /**
+   * Instantiate a accessor for the {@link Float8Vector}.
+   *
+   * @param vector             an instance of a Float8Vector.
+   * @param currentRowSupplier the supplier to track the lines.
+   * @param setCursorWasNull   the consumer to set if value was null.
+   */
+  public ArrowFlightJdbcFloat8VectorAccessor(Float8Vector vector,
+                                             IntSupplier currentRowSupplier,
+                                             ArrowFlightJdbcAccessorFactory.WasNullConsumer setCursorWasNull) {
+    super(currentRowSupplier, setCursorWasNull);
+    this.holder = new NullableFloat8Holder();
+    this.vector = vector;
+  }
+
+  @Override
+  public Class<?> getObjectClass() {
+    return Double.class;
+  }
+
+  @Override
+  public double getDouble() {
+    vector.get(getCurrentRow(), holder);
+
+    this.wasNull = holder.isSet == 0;
+    this.wasNullConsumer.setWasNull(this.wasNull);
+    if (this.wasNull) {
+      return 0;

Review comment:
       Wouldn't (signaling) NaN be more appropriate? Or is this what is expected from JDBC

##########
File path: java/flight/flight-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/accessor/impl/calendar/ArrowFlightJdbcIntervalVectorAccessorTest.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.driver.jdbc.accessor.impl.calendar;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+
+import java.time.Duration;
+import java.time.Period;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.function.Supplier;
+
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessorFactory;
+import org.apache.arrow.driver.jdbc.utils.AccessorTestUtils;
+import org.apache.arrow.driver.jdbc.utils.RootAllocatorTestRule;
+import org.apache.arrow.vector.IntervalDayVector;
+import org.apache.arrow.vector.IntervalYearVector;
+import org.apache.arrow.vector.ValueVector;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ErrorCollector;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class ArrowFlightJdbcIntervalVectorAccessorTest {
+
+  @ClassRule
+  public static RootAllocatorTestRule rootAllocatorTestRule = new RootAllocatorTestRule();
+
+  @Rule
+  public final ErrorCollector collector = new ErrorCollector();
+
+  private final Supplier<ValueVector> vectorSupplier;
+  private ValueVector vector;
+
+  private final AccessorTestUtils.AccessorSupplier<ArrowFlightJdbcIntervalVectorAccessor>
+      accessorSupplier = (vector, getCurrentRow) -> {
+        ArrowFlightJdbcAccessorFactory.WasNullConsumer noOpWasNullConsumer = (boolean wasNull) -> {
+        };
+        if (vector instanceof IntervalDayVector) {

Review comment:
       nit, but would it make sense to use the AccessorFactory here? ('here' being each of these test cases)

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/accessor/ArrowFlightJdbcAccessorFactory.java
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.driver.jdbc.accessor;
+
+import java.util.function.IntSupplier;
+
+import org.apache.arrow.driver.jdbc.accessor.impl.ArrowFlightJdbcNullVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.binary.ArrowFlightJdbcBinaryVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcDateVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcDurationVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcIntervalVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeStampVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcDenseUnionVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcFixedSizeListVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcLargeListVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcListVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcMapVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcStructVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.ArrowFlightJdbcUnionVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.numeric.ArrowFlightJdbcBaseIntVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.numeric.ArrowFlightJdbcBitVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.numeric.ArrowFlightJdbcDecimalVectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.numeric.ArrowFlightJdbcFloat4VectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.numeric.ArrowFlightJdbcFloat8VectorAccessor;
+import org.apache.arrow.driver.jdbc.accessor.impl.text.ArrowFlightJdbcVarCharVectorAccessor;
+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.DurationVector;
+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.IntervalDayVector;
+import org.apache.arrow.vector.IntervalYearVector;
+import org.apache.arrow.vector.LargeVarBinaryVector;
+import org.apache.arrow.vector.LargeVarCharVector;
+import org.apache.arrow.vector.NullVector;
+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.UInt1Vector;
+import org.apache.arrow.vector.UInt2Vector;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.arrow.vector.UInt8Vector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.complex.DenseUnionVector;
+import org.apache.arrow.vector.complex.FixedSizeListVector;
+import org.apache.arrow.vector.complex.LargeListVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.MapVector;
+import org.apache.arrow.vector.complex.StructVector;
+import org.apache.arrow.vector.complex.UnionVector;
+
+/**
+ * Factory to instantiate the accessors.
+ */
+public class ArrowFlightJdbcAccessorFactory {
+
+  /**
+   * Create an accessor according to the its type.
+   *
+   * @param vector        an instance of an arrow vector.
+   * @param getCurrentRow a supplier to check which row is being accessed.
+   * @return an instance of one of the accessors.
+   */
+  public static ArrowFlightJdbcAccessor createAccessor(ValueVector vector,
+                                                       IntSupplier getCurrentRow,
+                                                       WasNullConsumer setCursorWasNull) {
+    if (vector instanceof UInt1Vector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((UInt1Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof UInt2Vector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((UInt2Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof UInt4Vector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((UInt4Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof UInt8Vector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((UInt8Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof TinyIntVector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((TinyIntVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof SmallIntVector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((SmallIntVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof IntVector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((IntVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof BigIntVector) {
+      return new ArrowFlightJdbcBaseIntVectorAccessor((BigIntVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof Float4Vector) {
+      return new ArrowFlightJdbcFloat4VectorAccessor((Float4Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof Float8Vector) {
+      return new ArrowFlightJdbcFloat8VectorAccessor((Float8Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof BitVector) {
+      return new ArrowFlightJdbcBitVectorAccessor((BitVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof DecimalVector) {
+      return new ArrowFlightJdbcDecimalVectorAccessor((DecimalVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof Decimal256Vector) {
+      return new ArrowFlightJdbcDecimalVectorAccessor((Decimal256Vector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof VarBinaryVector) {
+      return new ArrowFlightJdbcBinaryVectorAccessor((VarBinaryVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof LargeVarBinaryVector) {
+      return new ArrowFlightJdbcBinaryVectorAccessor((LargeVarBinaryVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof FixedSizeBinaryVector) {
+      return new ArrowFlightJdbcBinaryVectorAccessor((FixedSizeBinaryVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof TimeStampVector) {
+      return new ArrowFlightJdbcTimeStampVectorAccessor((TimeStampVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof TimeNanoVector) {
+      return new ArrowFlightJdbcTimeVectorAccessor((TimeNanoVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof TimeMicroVector) {
+      return new ArrowFlightJdbcTimeVectorAccessor((TimeMicroVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof TimeMilliVector) {
+      return new ArrowFlightJdbcTimeVectorAccessor((TimeMilliVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof TimeSecVector) {
+      return new ArrowFlightJdbcTimeVectorAccessor((TimeSecVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof DateDayVector) {
+      return new ArrowFlightJdbcDateVectorAccessor(((DateDayVector) vector), getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof DateMilliVector) {
+      return new ArrowFlightJdbcDateVectorAccessor(((DateMilliVector) vector), getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof VarCharVector) {
+      return new ArrowFlightJdbcVarCharVectorAccessor((VarCharVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof LargeVarCharVector) {
+      return new ArrowFlightJdbcVarCharVectorAccessor((LargeVarCharVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof DurationVector) {
+      return new ArrowFlightJdbcDurationVectorAccessor((DurationVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof IntervalDayVector) {
+      return new ArrowFlightJdbcIntervalVectorAccessor(((IntervalDayVector) vector), getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof IntervalYearVector) {
+      return new ArrowFlightJdbcIntervalVectorAccessor(((IntervalYearVector) vector), getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof StructVector) {
+      return new ArrowFlightJdbcStructVectorAccessor((StructVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof MapVector) {
+      return new ArrowFlightJdbcMapVectorAccessor((MapVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof ListVector) {
+      return new ArrowFlightJdbcListVectorAccessor((ListVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof LargeListVector) {
+      return new ArrowFlightJdbcLargeListVectorAccessor((LargeListVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof FixedSizeListVector) {
+      return new ArrowFlightJdbcFixedSizeListVectorAccessor((FixedSizeListVector) vector,
+          getCurrentRow, setCursorWasNull);
+    } else if (vector instanceof UnionVector) {
+      return new ArrowFlightJdbcUnionVectorAccessor((UnionVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof DenseUnionVector) {
+      return new ArrowFlightJdbcDenseUnionVectorAccessor((DenseUnionVector) vector, getCurrentRow,
+          setCursorWasNull);
+    } else if (vector instanceof NullVector || vector == null) {

Review comment:
       Can a null vector ever occur?

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcFactory.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.driver.jdbc;
+
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.Properties;
+import java.util.TimeZone;
+
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaFactory;
+import org.apache.calcite.avatica.AvaticaResultSetMetaData;
+import org.apache.calcite.avatica.AvaticaSpecificDatabaseMetaData;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.QueryState;
+import org.apache.calcite.avatica.UnregisteredDriver;
+
+/**
+ * Factory for the Arrow Flight JDBC Driver.
+ */
+public class ArrowFlightJdbcFactory implements AvaticaFactory {

Review comment:
       nit: sorry, but…these are all "Flight SQL" drivers/factories/connections/etc. not just "Flight" right?

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcDriver.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.driver.jdbc;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.org.apache.http.NameValuePair;
+import org.apache.calcite.avatica.org.apache.http.client.utils.URLEncodedUtils;
+
+/**
+ * JDBC driver for querying data from an Apache Arrow Flight server.
+ */
+public class ArrowFlightJdbcDriver extends UnregisteredDriver {
+
+  private static final String CONNECT_STRING_PREFIX = "jdbc:arrow-flight://";
+  private static DriverVersion version;
+
+  static {
+    // Special code for supporting Java9 and higher.
+    // Netty requires some extra properties to unlock some native memory management api
+    // Setting this property if not already set externally
+    // This has to be done before any netty class is being loaded
+    final String key = "cfjd.io.netty.tryReflectionSetAccessible";
+    final String tryReflectionSetAccessible = System.getProperty(key);
+    if (tryReflectionSetAccessible == null) {
+      System.setProperty(key, Boolean.TRUE.toString());
+    }
+
+    new ArrowFlightJdbcDriver().register();
+  }
+
+  @Override
+  public ArrowFlightConnection connect(final String url, final Properties info)
+      throws SQLException {
+    final Properties properties = new Properties(info);
+    properties.putAll(info);
+
+    if (url != null) {
+      final Map<Object, Object> propertiesFromUrl = getUrlsArgs(url);
+      properties.putAll(propertiesFromUrl);
+    }
+
+    try {
+      return ArrowFlightConnection.createNewConnection(
+          this,
+          factory,
+          url,
+          properties,
+          new RootAllocator(Long.MAX_VALUE));
+    } catch (AssertionError | FlightRuntimeException e) {
+      throw new SQLException("Failed to connect.", e);
+    }
+  }
+
+  @Override
+  protected String getFactoryClassName(final JdbcVersion jdbcVersion) {
+    return ArrowFlightJdbcFactory.class.getName();
+  }
+
+  @Override
+  protected DriverVersion createDriverVersion() {
+
+    CreateVersionIfNull:
+    {
+
+      if (version != null) {
+        break CreateVersionIfNull;
+      }
+
+      try (Reader reader = new BufferedReader(new InputStreamReader(
+          this.getClass().getResourceAsStream("/properties/flight.properties"),
+          StandardCharsets.UTF_8))) {
+        final Properties properties = new Properties();
+        properties.load(reader);
+
+        final String parentName = properties
+            .getProperty("org.apache.arrow.flight.name");
+        final String parentVersion = properties
+            .getProperty("org.apache.arrow.flight.version");
+        final String[] pVersion = parentVersion.split("\\.");
+
+        final int parentMajorVersion = Integer.parseInt(pVersion[0]);
+        final int parentMinorVersion = Integer.parseInt(pVersion[1]);
+
+        final String childName = properties
+            .getProperty("org.apache.arrow.flight.jdbc-driver.name");
+        final String childVersion = properties
+            .getProperty("org.apache.arrow.flight.jdbc-driver.version");
+        final String[] cVersion = childVersion.split("\\.");
+
+        final int childMajorVersion = Integer.parseInt(cVersion[0]);
+        final int childMinorVersion = Integer.parseInt(cVersion[1]);
+
+        version = new DriverVersion(childName, childVersion, parentName,
+            parentVersion, true, childMajorVersion, childMinorVersion,
+            parentMajorVersion, parentMinorVersion);
+      } catch (final IOException e) {
+        throw new RuntimeException("Failed to load driver version.", e);
+      }
+    }
+
+    return version;
+  }
+
+  @Override
+  public Meta createMeta(final AvaticaConnection connection) {
+    return new ArrowFlightMetaImpl((ArrowFlightConnection) connection);
+  }
+
+  @Override
+  protected String getConnectStringPrefix() {
+    return CONNECT_STRING_PREFIX;
+  }
+
+  @Override
+  public boolean acceptsURL(final String url) throws SQLException {
+    return Preconditions.checkNotNull(url).startsWith(CONNECT_STRING_PREFIX);
+  }
+
+  /**
+   * Parses the provided url based on the format this driver accepts, retrieving
+   * arguments after the {@link #CONNECT_STRING_PREFIX}.
+   * <p>
+   * This method gets the args if the provided URL follows this pattern:
+   * {@code jdbc:arrow-flight://<host>:<port>[/?key1=val1&key2=val2&(...)]}
+   *
+   * <table border="1">
+   *    <tr>
+   *        <td>Group</td>
+   *        <td>Definition</td>
+   *        <td>Value</td>
+   *    </tr>
+   *    <tr>
+   *        <td>? — inaccessible</td>
+   *        <td>{@link #getConnectStringPrefix}</td>
+   *        <td>
+   *            the URL prefix accepted by this driver, i.e.,
+   *            {@code "jdbc:arrow-flight://"}
+   *        </td>
+   *    </tr>
+   *    <tr>
+   *        <td>1</td>
+   *        <td>IPv4 host name</td>
+   *        <td>
+   *            first word after previous group and before "{@code :}"
+   *        </td>
+   *    </tr>
+   *    <tr>
+   *        <td>2</td>
+   *        <td>IPv4 port number</td>
+   *        <td>
+   *            first number after previous group and before "{@code /?}"
+   *        </td>
+   *    </tr>
+   *    <tr>
+   *        <td>3</td>
+   *        <td>custom call parameters</td>
+   *        <td>
+   *            all parameters provided after "{@code /?}" — must follow the
+   *            pattern: "{@code key=value}" with "{@code &}" separating a
+   *            parameter from another
+   *        </td>
+   *    </tr>
+   * </table>
+   *
+   * @param url The url to parse.
+   * @return the parsed arguments.
+   * @throws SQLException If an error occurs while trying to parse the URL.
+   */
+  private Map<Object, Object> getUrlsArgs(String url)
+      throws SQLException {
+
+    /*
+     *
+     * Perhaps this logic should be inside a utility class, separated from this
+     * one, so as to better delegate responsibilities and concerns throughout
+     * the code and increase maintainability.
+     *
+     * =====
+     *
+     * Keep in mind that the URL must ALWAYS follow the pattern:
+     * "jdbc:arrow-flight://<host>:<port>[/?param1=value1&param2=value2&(...)]."
+     *
+     */
+
+    if (!url.startsWith("jdbc:")) {
+      throw new SQLException("Malformed/invalid URL!");
+    }
+
+    // It's necessary to use a string without "jdbc:" at the beginning to be parsed as a valid URL.
+    url = url.substring(5);
+
+    final URI uri;
+
+    try {
+      uri = URI.create(url);
+    } catch (final IllegalArgumentException e) {
+      throw new SQLException("Malformed/invalid URL!", e);
+    }
+
+    if (!Objects.equals(uri.getScheme(), "arrow-flight")) {

Review comment:
       How do we distinguish between Flight schemes (e.g. 'grpc', 'grpc+tls', 'grpc+unix', eventually 'ucx', etc.)?

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcDriver.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.driver.jdbc;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.org.apache.http.NameValuePair;
+import org.apache.calcite.avatica.org.apache.http.client.utils.URLEncodedUtils;
+
+/**
+ * JDBC driver for querying data from an Apache Arrow Flight server.
+ */
+public class ArrowFlightJdbcDriver extends UnregisteredDriver {
+
+  private static final String CONNECT_STRING_PREFIX = "jdbc:arrow-flight://";
+  private static DriverVersion version;
+
+  static {
+    // Special code for supporting Java9 and higher.
+    // Netty requires some extra properties to unlock some native memory management api
+    // Setting this property if not already set externally
+    // This has to be done before any netty class is being loaded
+    final String key = "cfjd.io.netty.tryReflectionSetAccessible";
+    final String tryReflectionSetAccessible = System.getProperty(key);
+    if (tryReflectionSetAccessible == null) {
+      System.setProperty(key, Boolean.TRUE.toString());
+    }

Review comment:
       Do we want to force this here? It looks like we're shading netty…is this all for distribution purposes then?

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/utils/ExceptionTemplateThrower.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.driver.jdbc.utils;
+
+import static java.lang.String.format;
+
+import org.apache.calcite.avatica.util.Cursor.Accessor;
+
+/**
+ * Utility class for managing exceptions thrown by
+ * {@link Accessor}s.
+ */
+public final class ExceptionTemplateThrower {
+
+  private ExceptionTemplateThrower() {
+    // Prevent instantiation.
+  }
+
+  /**
+   * Gets a {@link Exception} for an attempt to perform a conversion
+   * not yet supported by the {@link Accessor} in use.
+   *
+   * @return the exception.
+   */
+  public static UnsupportedOperationException getOperationNotSupported(final Class<?> type) {

Review comment:
       It seems this is used purely in the base Accessor class, maybe it could be a private static method of that class?

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcDriver.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.driver.jdbc;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.org.apache.http.NameValuePair;
+import org.apache.calcite.avatica.org.apache.http.client.utils.URLEncodedUtils;
+
+/**
+ * JDBC driver for querying data from an Apache Arrow Flight server.
+ */
+public class ArrowFlightJdbcDriver extends UnregisteredDriver {
+
+  private static final String CONNECT_STRING_PREFIX = "jdbc:arrow-flight://";
+  private static DriverVersion version;
+
+  static {
+    // Special code for supporting Java9 and higher.
+    // Netty requires some extra properties to unlock some native memory management api
+    // Setting this property if not already set externally
+    // This has to be done before any netty class is being loaded
+    final String key = "cfjd.io.netty.tryReflectionSetAccessible";
+    final String tryReflectionSetAccessible = System.getProperty(key);
+    if (tryReflectionSetAccessible == null) {
+      System.setProperty(key, Boolean.TRUE.toString());
+    }
+
+    new ArrowFlightJdbcDriver().register();
+  }
+
+  @Override
+  public ArrowFlightConnection connect(final String url, final Properties info)
+      throws SQLException {
+    final Properties properties = new Properties(info);
+    properties.putAll(info);
+
+    if (url != null) {
+      final Map<Object, Object> propertiesFromUrl = getUrlsArgs(url);
+      properties.putAll(propertiesFromUrl);
+    }
+
+    try {
+      return ArrowFlightConnection.createNewConnection(
+          this,
+          factory,
+          url,
+          properties,
+          new RootAllocator(Long.MAX_VALUE));
+    } catch (AssertionError | FlightRuntimeException e) {

Review comment:
       Why are we catching AssertionError? It seems we should be catching SQLException instead?

##########
File path: java/flight/flight-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/utils/UrlSample.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.driver.jdbc.utils;
+
+import org.apache.arrow.util.Preconditions;
+
+/**
+ * Class for storing sample JDBC URLs. Used for testing.
+ *
+ * @see org.apache.arrow.driver.jdbc.utils.BaseProperty
+ * @deprecated not updatable to match dinamic server allocation.
+ */
+@Deprecated
+public enum UrlSample {

Review comment:
       Why are we adding a new deprecated class?

##########
File path: java/flight/flight-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/utils/PropertiesSample.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.driver.jdbc.utils;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.Properties;
+
+import javax.annotation.Nullable;
+
+import org.apache.arrow.driver.jdbc.FlightServerTestRule;
+import org.apache.arrow.util.Preconditions;
+
+
+/**
+ * {@link Properties} wrapper used for testing. Uses sample values.
+ *
+ * @see FlightServerTestRule
+ * @deprecated not updatable to match dinamic server allocation.
+ */
+@Deprecated

Review comment:
       Ditto here, why add a class only to immediately deprecate it? (I guess things are in the middle of a refactoring here?)

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightConnection.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.driver.jdbc;
+
+import java.sql.SQLException;
+import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.arrow.driver.jdbc.client.ArrowFlightSqlClientHandler;
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl;
+import org.apache.arrow.flight.FlightClient;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.util.AutoCloseables;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaFactory;
+
+import io.netty.util.concurrent.DefaultThreadFactory;
+
+/**
+ * Connection to the Arrow Flight server.
+ */
+public final class ArrowFlightConnection extends AvaticaConnection {
+
+  private final BufferAllocator allocator;
+  private final ArrowFlightSqlClientHandler clientHandler;
+  private final ArrowFlightConnectionConfigImpl config;
+  private ExecutorService executorService;
+
+  /**
+   * Creates a new {@link ArrowFlightConnection}.
+   *
+   * @param driver        the {@link ArrowFlightJdbcDriver} to use.
+   * @param factory       the {@link AvaticaFactory} to use.
+   * @param url           the URL to use.
+   * @param properties    the {@link Properties} to use.
+   * @param config        the {@link ArrowFlightConnectionConfigImpl} to use.
+   * @param allocator     the {@link BufferAllocator} to use.
+   * @param clientHandler the {@link ArrowFlightSqlClientHandler} to use.
+   */
+  private ArrowFlightConnection(final ArrowFlightJdbcDriver driver, final AvaticaFactory factory,
+                                final String url, final Properties properties,
+                                final ArrowFlightConnectionConfigImpl config,
+                                final BufferAllocator allocator,
+                                final ArrowFlightSqlClientHandler clientHandler) {
+    super(driver, factory, url, properties);
+    this.config = Preconditions.checkNotNull(config, "Config cannot be null.");
+    this.allocator = Preconditions.checkNotNull(allocator, "Allocator cannot be null.");
+    this.clientHandler = Preconditions.checkNotNull(clientHandler, "Handler cannot be null.");
+  }
+
+  /**
+   * Creates a new {@link ArrowFlightConnection} to a {@link FlightClient}.
+   *
+   * @param driver     the {@link ArrowFlightJdbcDriver} to use.
+   * @param factory    the {@link AvaticaFactory} to use.
+   * @param url        the URL to establish the connection to.
+   * @param properties the {@link Properties} to use for this session.
+   * @param allocator  the {@link BufferAllocator} to use.
+   * @return a new {@link ArrowFlightConnection}.
+   * @throws SQLException on error.
+   */
+  static ArrowFlightConnection createNewConnection(final ArrowFlightJdbcDriver driver,
+                                                   final AvaticaFactory factory,
+                                                   final String url, final Properties properties,
+                                                   final BufferAllocator allocator)
+      throws SQLException {
+    final ArrowFlightConnectionConfigImpl config = new ArrowFlightConnectionConfigImpl(properties);
+    final ArrowFlightSqlClientHandler clientHandler = createNewClientHandler(config, allocator);
+    return new ArrowFlightConnection(driver, factory, url, properties, config, allocator,
+        clientHandler);
+  }
+
+  private static ArrowFlightSqlClientHandler createNewClientHandler(
+      final ArrowFlightConnectionConfigImpl config,
+      final BufferAllocator allocator) throws SQLException {
+    try {
+      return new ArrowFlightSqlClientHandler.Builder()
+          .withHost(config.getHost())
+          .withPort(config.getPort())
+          .withUsername(config.getUser())
+          .withPassword(config.getPassword())
+          .withKeyStorePath(config.getKeyStorePath())
+          .withKeyStorePassword(config.keystorePassword())
+          .withBufferAllocator(allocator)
+          .withTlsEncryption(config.useTls())
+          .withCallOptions(config.toCallOption())
+          .build();
+    } catch (final SQLException e) {
+      allocator.close();

Review comment:
       If `allocator.close` throws we might want to add `e` as a suppressed exception

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcPooledConnection.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.driver.jdbc;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+import javax.sql.ConnectionEvent;
+import javax.sql.ConnectionEventListener;
+import javax.sql.PooledConnection;
+import javax.sql.StatementEventListener;
+
+import org.apache.arrow.driver.jdbc.utils.ConnectionWrapper;
+
+/**
+ * {@link PooledConnection} implementation for Arrow Flight JDBC Driver.
+ */
+public class ArrowFlightJdbcPooledConnection implements PooledConnection {
+
+  private final ArrowFlightConnection connection;
+  private final Set<ConnectionEventListener> eventListeners;
+  private final Set<StatementEventListener> statementEventListeners;
+
+  private final class ConnectionHandle extends ConnectionWrapper {

Review comment:
       The purpose of this class is to return the connection to the pool when closed?

##########
File path: java/flight/flight-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/utils/FlightStreamQueueTest.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.driver.jdbc.utils;
+
+import static java.lang.String.format;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.mockito.Mockito.mock;
+
+import java.util.Optional;
+import java.util.concurrent.CompletionService;
+
+import org.apache.arrow.flight.FlightStream;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ErrorCollector;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+/**
+ * Tests for {@link FlightStreamQueue}.
+ */
+@RunWith(MockitoJUnitRunner.class)
+public class FlightStreamQueueTest {
+
+  @Rule
+  public final ErrorCollector collector = new ErrorCollector();
+  @Mock
+  private CompletionService<FlightStream> mockedService;
+  private FlightStreamQueue queue;
+
+  @Before
+  public void setUp() {
+    queue = new FlightStreamQueue(mockedService);
+  }
+
+  @Test
+  public void testNextShouldRetrieveNullIfEmpty() throws Exception {
+    collector.checkThat(queue.next(), is(nullValue()));
+  }
+
+  @Test
+  public void testNextShouldThrowExceptionUponClose() throws Exception {
+    queue.close();
+    Optional<Exception> expectedExceptionOnNextIfClosed = Optional.empty();
+    try {
+      queue.next();
+    } catch (final IllegalStateException e) {
+      expectedExceptionOnNextIfClosed = Optional.of(e);

Review comment:
       It seems other parts of the codebase use assertThrows for this type of check

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java
##########
@@ -0,0 +1,516 @@
+/*
+ * 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.driver.jdbc.client;
+
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.arrow.driver.jdbc.client.utils.ClientAuthenticationUtils;
+import org.apache.arrow.flight.CallOption;
+import org.apache.arrow.flight.FlightClient;
+import org.apache.arrow.flight.FlightClientMiddleware;
+import org.apache.arrow.flight.FlightEndpoint;
+import org.apache.arrow.flight.FlightInfo;
+import org.apache.arrow.flight.FlightStream;
+import org.apache.arrow.flight.Location;
+import org.apache.arrow.flight.auth2.ClientBearerHeaderHandler;
+import org.apache.arrow.flight.auth2.ClientIncomingAuthHeaderMiddleware;
+import org.apache.arrow.flight.sql.FlightSqlClient;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlInfo;
+import org.apache.arrow.flight.sql.util.TableRef;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.util.AutoCloseables;
+import org.apache.arrow.util.Preconditions;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.calcite.avatica.Meta.StatementType;
+
+/**
+ * A {@link FlightSqlClient} handler.
+ */
+public final class ArrowFlightSqlClientHandler implements AutoCloseable {
+
+  private final FlightSqlClient sqlClient;
+  private final Set<CallOption> options = new HashSet<>();
+
+  ArrowFlightSqlClientHandler(final FlightSqlClient sqlClient,
+                              final Collection<CallOption> options) {
+    this.options.addAll(options);
+    this.sqlClient = Preconditions.checkNotNull(sqlClient);
+  }
+
+  /**
+   * Creates a new {@link ArrowFlightSqlClientHandler} from the provided {@code client} and {@code options}.
+   *
+   * @param client  the {@link FlightClient} to manage under a {@link FlightSqlClient} wrapper.
+   * @param options the {@link CallOption}s to persist in between subsequent client calls.
+   * @return a new {@link ArrowFlightSqlClientHandler}.
+   */
+  public static ArrowFlightSqlClientHandler createNewHandler(final FlightClient client,
+                                                             final Collection<CallOption> options) {
+    return new ArrowFlightSqlClientHandler(new FlightSqlClient(client), options);
+  }
+
+  /**
+   * Gets the {@link #options} for the subsequent calls from this handler.
+   *
+   * @return the {@link CallOption}s.
+   */
+  private CallOption[] getOptions() {
+    return options.toArray(new CallOption[0]);
+  }
+
+  /**
+   * Makes an RPC "getStream" request based on the provided {@link FlightInfo}
+   * object. Retrieves the result of the query previously prepared with "getInfo."
+   *
+   * @param flightInfo The {@link FlightInfo} instance from which to fetch results.
+   * @return a {@code FlightStream} of results.
+   */
+  public List<FlightStream> getStreams(final FlightInfo flightInfo) {
+    return flightInfo.getEndpoints().stream()
+        .map(FlightEndpoint::getTicket)
+        .map(ticket -> sqlClient.getStream(ticket, getOptions()))
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * Makes an RPC "getInfo" request based on the provided {@code query}
+   * object.
+   *
+   * @param query The query.
+   * @return a {@code FlightStream} of results.
+   */
+  public FlightInfo getInfo(final String query) {
+    return sqlClient.execute(query, getOptions());
+  }
+
+  @Override
+  public void close() throws SQLException {
+    try {
+      AutoCloseables.close(sqlClient);
+    } catch (final Exception e) {
+      throw new SQLException("Failed to clean up client resources.", e);
+    }
+  }
+
+  /**
+   * A prepared statement handler.
+   */
+  public interface PreparedStatement extends AutoCloseable {
+    /**
+     * Executes this {@link PreparedStatement}.
+     *
+     * @return the {@link FlightInfo} representing the outcome of this query execution.
+     * @throws SQLException on error.
+     */
+    FlightInfo executeQuery() throws SQLException;
+
+    /**
+     * Executes a {@link StatementType#UPDATE} query.
+     *
+     * @return the number of rows affected.
+     */
+    long executeUpdate();
+
+    /**
+     * Gets the {@link StatementType} of this {@link PreparedStatement}.
+     *
+     * @return the Statement Type.
+     */
+    StatementType getType();
+
+    /**
+     * Gets the {@link Schema} of this {@link PreparedStatement}.
+     *
+     * @return {@link Schema}.
+     */
+    Schema getDataSetSchema();
+
+    @Override
+    void close();
+  }
+
+  /**
+   * Creates a new {@link PreparedStatement} for the given {@code query}.
+   *
+   * @param query the SQL query.
+   * @return a new prepared statement.
+   */
+  public PreparedStatement prepare(final String query) {
+    final FlightSqlClient.PreparedStatement preparedStatement =
+        sqlClient.prepare(query, getOptions());
+    return new PreparedStatement() {
+      @Override
+      public FlightInfo executeQuery() throws SQLException {
+        return preparedStatement.execute(getOptions());

Review comment:
       And in general most of the calls to FlightSqlClient here should.

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcDriver.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.driver.jdbc;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.org.apache.http.NameValuePair;
+import org.apache.calcite.avatica.org.apache.http.client.utils.URLEncodedUtils;
+
+/**
+ * JDBC driver for querying data from an Apache Arrow Flight server.
+ */
+public class ArrowFlightJdbcDriver extends UnregisteredDriver {
+
+  private static final String CONNECT_STRING_PREFIX = "jdbc:arrow-flight://";
+  private static DriverVersion version;
+
+  static {
+    // Special code for supporting Java9 and higher.
+    // Netty requires some extra properties to unlock some native memory management api
+    // Setting this property if not already set externally
+    // This has to be done before any netty class is being loaded
+    final String key = "cfjd.io.netty.tryReflectionSetAccessible";
+    final String tryReflectionSetAccessible = System.getProperty(key);
+    if (tryReflectionSetAccessible == null) {
+      System.setProperty(key, Boolean.TRUE.toString());
+    }

Review comment:
       Also in that case, for Java 11+ there's more flags that need to be set, right? Are those documented?

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/accessor/impl/calendar/ArrowFlightJdbcDateVectorGetter.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.driver.jdbc.accessor.impl.calendar;
+
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DateMilliVector;
+import org.apache.arrow.vector.holders.NullableDateDayHolder;
+import org.apache.arrow.vector.holders.NullableDateMilliHolder;
+
+/**
+ * Auxiliary class used to unify data access on TimeStampVectors.
+ */
+final class ArrowFlightJdbcDateVectorGetter {
+
+  private ArrowFlightJdbcDateVectorGetter() {
+    // Prevent instantiation.
+  }
+
+  /**
+   * Auxiliary class meant to unify Date*Vector#get implementations with different classes of ValueHolders.
+   */
+  static class Holder {
+    int isSet;
+    long value;

Review comment:
       Can we document what `value` is (e.g. the unit - I suppose this is just "whatever the vector's unit was")?

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcDriver.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.driver.jdbc;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.org.apache.http.NameValuePair;
+import org.apache.calcite.avatica.org.apache.http.client.utils.URLEncodedUtils;
+
+/**
+ * JDBC driver for querying data from an Apache Arrow Flight server.
+ */
+public class ArrowFlightJdbcDriver extends UnregisteredDriver {
+
+  private static final String CONNECT_STRING_PREFIX = "jdbc:arrow-flight://";
+  private static DriverVersion version;
+
+  static {
+    // Special code for supporting Java9 and higher.
+    // Netty requires some extra properties to unlock some native memory management api
+    // Setting this property if not already set externally
+    // This has to be done before any netty class is being loaded
+    final String key = "cfjd.io.netty.tryReflectionSetAccessible";
+    final String tryReflectionSetAccessible = System.getProperty(key);
+    if (tryReflectionSetAccessible == null) {
+      System.setProperty(key, Boolean.TRUE.toString());
+    }
+
+    new ArrowFlightJdbcDriver().register();
+  }
+
+  @Override
+  public ArrowFlightConnection connect(final String url, final Properties info)
+      throws SQLException {
+    final Properties properties = new Properties(info);
+    properties.putAll(info);
+
+    if (url != null) {
+      final Map<Object, Object> propertiesFromUrl = getUrlsArgs(url);
+      properties.putAll(propertiesFromUrl);
+    }
+
+    try {
+      return ArrowFlightConnection.createNewConnection(
+          this,
+          factory,
+          url,
+          properties,
+          new RootAllocator(Long.MAX_VALUE));
+    } catch (AssertionError | FlightRuntimeException e) {

Review comment:
       Also, I think it would be very surprising to users if we bubble a FlightRuntimeException up without wrapping it in a SQLException.

##########
File path: java/flight/flight-jdbc-driver/src/test/resources/keys/cert0.pem
##########
@@ -0,0 +1,29 @@
+-----BEGIN CERTIFICATE-----

Review comment:
       We have certificates in /testing/data/flight, could those be used? (I guess we need to make a keystore for them.)

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcFactory.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.driver.jdbc;
+
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.Properties;
+import java.util.TimeZone;
+
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaFactory;
+import org.apache.calcite.avatica.AvaticaResultSetMetaData;
+import org.apache.calcite.avatica.AvaticaSpecificDatabaseMetaData;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.QueryState;
+import org.apache.calcite.avatica.UnregisteredDriver;
+
+/**
+ * Factory for the Arrow Flight JDBC Driver.
+ */
+public class ArrowFlightJdbcFactory implements AvaticaFactory {
+  private final int major;
+  private final int minor;
+
+  // This need to be public so Avatica can call this constructor
+  public ArrowFlightJdbcFactory() {
+    this(4, 1);
+  }
+
+  private ArrowFlightJdbcFactory(final int major, final int minor) {
+    this.major = major;
+    this.minor = minor;
+  }
+
+  @Override
+  public AvaticaConnection newConnection(final UnregisteredDriver driver,
+                                         final AvaticaFactory factory,
+                                         final String url,
+                                         final Properties info) throws SQLException {
+    return ArrowFlightConnection.createNewConnection(
+        (ArrowFlightJdbcDriver) driver,
+        factory,
+        url,
+        info,
+        new RootAllocator(Long.MAX_VALUE));

Review comment:
       This allocator is closed in ArrowFlightConnection::close it seems? Is it worth providing a way for the user to pass in an allocator? Also, we may want to name the allocator just to aid debugging.

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/accessor/impl/text/ArrowFlightJdbcVarCharVectorAccessor.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.driver.jdbc.accessor.impl.text;
+
+import java.io.ByteArrayInputStream;
+import java.io.CharArrayReader;
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Calendar;
+import java.util.function.IntSupplier;
+
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessor;
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessorFactory;
+import org.apache.arrow.driver.jdbc.utils.DateTimeUtils;
+import org.apache.arrow.vector.LargeVarCharVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.util.Text;
+
+/**
+ * Accessor for the Arrow types: {@link VarCharVector} and {@link LargeVarCharVector}.
+ */
+public class ArrowFlightJdbcVarCharVectorAccessor extends ArrowFlightJdbcAccessor {
+
+  /**
+   * Functional interface to help integrating VarCharVector and LargeVarCharVector.
+   */
+  @FunctionalInterface
+  interface Getter {
+    Text get(int index);
+  }
+
+  private final Getter getter;
+
+  public ArrowFlightJdbcVarCharVectorAccessor(VarCharVector vector,
+                                              IntSupplier currentRowSupplier,
+                                              ArrowFlightJdbcAccessorFactory.WasNullConsumer setCursorWasNull) {
+    this(vector::getObject, currentRowSupplier, setCursorWasNull);
+  }
+
+  public ArrowFlightJdbcVarCharVectorAccessor(LargeVarCharVector vector,
+                                              IntSupplier currentRowSupplier,
+                                              ArrowFlightJdbcAccessorFactory.WasNullConsumer setCursorWasNull) {
+    this(vector::getObject, currentRowSupplier, setCursorWasNull);
+  }
+
+  ArrowFlightJdbcVarCharVectorAccessor(Getter getter,
+                                       IntSupplier currentRowSupplier,
+                                       ArrowFlightJdbcAccessorFactory.WasNullConsumer setCursorWasNull) {
+    super(currentRowSupplier, setCursorWasNull);
+    this.getter = getter;
+  }
+
+  @Override
+  public Class<?> getObjectClass() {
+    return String.class;
+  }
+
+  private Text getText() {
+    final Text text = this.getter.get(getCurrentRow());
+    this.wasNull = text == null;
+    this.wasNullConsumer.setWasNull(this.wasNull);
+    return text;
+  }
+
+  @Override
+  public String getObject() {
+    final Text text = getText();
+    return text == null ? null : text.toString();
+  }
+
+  @Override
+  public String getString() {
+    return getObject();
+  }
+
+  @Override
+  public byte[] getBytes() {
+    final Text value = this.getText();
+    return value == null ? null : value.copyBytes();
+  }
+
+  @Override
+  public boolean getBoolean() {

Review comment:
       Is this actually expected to be implemented of a string column?

##########
File path: java/flight/flight-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/adhoc/MockFlightSqlProducer.java
##########
@@ -0,0 +1,525 @@
+/*
+ * 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.driver.jdbc.adhoc;

Review comment:
       Why are some test utilities under 'adhoc' and others under 'utils'?

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcDriver.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.driver.jdbc;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.org.apache.http.NameValuePair;
+import org.apache.calcite.avatica.org.apache.http.client.utils.URLEncodedUtils;
+
+/**
+ * JDBC driver for querying data from an Apache Arrow Flight server.
+ */
+public class ArrowFlightJdbcDriver extends UnregisteredDriver {
+
+  private static final String CONNECT_STRING_PREFIX = "jdbc:arrow-flight://";
+  private static DriverVersion version;
+
+  static {
+    // Special code for supporting Java9 and higher.
+    // Netty requires some extra properties to unlock some native memory management api
+    // Setting this property if not already set externally
+    // This has to be done before any netty class is being loaded
+    final String key = "cfjd.io.netty.tryReflectionSetAccessible";
+    final String tryReflectionSetAccessible = System.getProperty(key);
+    if (tryReflectionSetAccessible == null) {
+      System.setProperty(key, Boolean.TRUE.toString());
+    }
+
+    new ArrowFlightJdbcDriver().register();
+  }
+
+  @Override
+  public ArrowFlightConnection connect(final String url, final Properties info)
+      throws SQLException {
+    final Properties properties = new Properties(info);
+    properties.putAll(info);
+
+    if (url != null) {
+      final Map<Object, Object> propertiesFromUrl = getUrlsArgs(url);
+      properties.putAll(propertiesFromUrl);
+    }
+
+    try {
+      return ArrowFlightConnection.createNewConnection(
+          this,
+          factory,
+          url,
+          properties,
+          new RootAllocator(Long.MAX_VALUE));
+    } catch (AssertionError | FlightRuntimeException e) {
+      throw new SQLException("Failed to connect.", e);
+    }
+  }
+
+  @Override
+  protected String getFactoryClassName(final JdbcVersion jdbcVersion) {
+    return ArrowFlightJdbcFactory.class.getName();
+  }
+
+  @Override
+  protected DriverVersion createDriverVersion() {
+
+    CreateVersionIfNull:
+    {
+
+      if (version != null) {
+        break CreateVersionIfNull;
+      }
+
+      try (Reader reader = new BufferedReader(new InputStreamReader(
+          this.getClass().getResourceAsStream("/properties/flight.properties"),

Review comment:
       This is generated by Maven then? (I couldn't find the file itself)

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcDriver.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.driver.jdbc;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.org.apache.http.NameValuePair;
+import org.apache.calcite.avatica.org.apache.http.client.utils.URLEncodedUtils;
+
+/**
+ * JDBC driver for querying data from an Apache Arrow Flight server.
+ */
+public class ArrowFlightJdbcDriver extends UnregisteredDriver {
+
+  private static final String CONNECT_STRING_PREFIX = "jdbc:arrow-flight://";
+  private static DriverVersion version;
+
+  static {
+    // Special code for supporting Java9 and higher.
+    // Netty requires some extra properties to unlock some native memory management api
+    // Setting this property if not already set externally
+    // This has to be done before any netty class is being loaded
+    final String key = "cfjd.io.netty.tryReflectionSetAccessible";
+    final String tryReflectionSetAccessible = System.getProperty(key);
+    if (tryReflectionSetAccessible == null) {
+      System.setProperty(key, Boolean.TRUE.toString());
+    }
+
+    new ArrowFlightJdbcDriver().register();
+  }
+
+  @Override
+  public ArrowFlightConnection connect(final String url, final Properties info)
+      throws SQLException {
+    final Properties properties = new Properties(info);
+    properties.putAll(info);
+
+    if (url != null) {
+      final Map<Object, Object> propertiesFromUrl = getUrlsArgs(url);
+      properties.putAll(propertiesFromUrl);
+    }
+
+    try {
+      return ArrowFlightConnection.createNewConnection(
+          this,
+          factory,
+          url,
+          properties,
+          new RootAllocator(Long.MAX_VALUE));
+    } catch (AssertionError | FlightRuntimeException e) {
+      throw new SQLException("Failed to connect.", e);
+    }
+  }
+
+  @Override
+  protected String getFactoryClassName(final JdbcVersion jdbcVersion) {
+    return ArrowFlightJdbcFactory.class.getName();
+  }
+
+  @Override
+  protected DriverVersion createDriverVersion() {
+
+    CreateVersionIfNull:
+    {
+
+      if (version != null) {
+        break CreateVersionIfNull;
+      }
+
+      try (Reader reader = new BufferedReader(new InputStreamReader(
+          this.getClass().getResourceAsStream("/properties/flight.properties"),
+          StandardCharsets.UTF_8))) {
+        final Properties properties = new Properties();
+        properties.load(reader);
+
+        final String parentName = properties
+            .getProperty("org.apache.arrow.flight.name");
+        final String parentVersion = properties
+            .getProperty("org.apache.arrow.flight.version");
+        final String[] pVersion = parentVersion.split("\\.");
+
+        final int parentMajorVersion = Integer.parseInt(pVersion[0]);
+        final int parentMinorVersion = Integer.parseInt(pVersion[1]);
+
+        final String childName = properties
+            .getProperty("org.apache.arrow.flight.jdbc-driver.name");
+        final String childVersion = properties
+            .getProperty("org.apache.arrow.flight.jdbc-driver.version");
+        final String[] cVersion = childVersion.split("\\.");
+
+        final int childMajorVersion = Integer.parseInt(cVersion[0]);
+        final int childMinorVersion = Integer.parseInt(cVersion[1]);
+
+        version = new DriverVersion(childName, childVersion, parentName,
+            parentVersion, true, childMajorVersion, childMinorVersion,
+            parentMajorVersion, parentMinorVersion);
+      } catch (final IOException e) {
+        throw new RuntimeException("Failed to load driver version.", e);
+      }
+    }
+
+    return version;
+  }
+
+  @Override
+  public Meta createMeta(final AvaticaConnection connection) {
+    return new ArrowFlightMetaImpl((ArrowFlightConnection) connection);
+  }
+
+  @Override
+  protected String getConnectStringPrefix() {
+    return CONNECT_STRING_PREFIX;
+  }
+
+  @Override
+  public boolean acceptsURL(final String url) throws SQLException {
+    return Preconditions.checkNotNull(url).startsWith(CONNECT_STRING_PREFIX);
+  }
+
+  /**
+   * Parses the provided url based on the format this driver accepts, retrieving
+   * arguments after the {@link #CONNECT_STRING_PREFIX}.
+   * <p>
+   * This method gets the args if the provided URL follows this pattern:
+   * {@code jdbc:arrow-flight://<host>:<port>[/?key1=val1&key2=val2&(...)]}
+   *
+   * <table border="1">
+   *    <tr>
+   *        <td>Group</td>
+   *        <td>Definition</td>
+   *        <td>Value</td>
+   *    </tr>
+   *    <tr>
+   *        <td>? — inaccessible</td>
+   *        <td>{@link #getConnectStringPrefix}</td>
+   *        <td>
+   *            the URL prefix accepted by this driver, i.e.,
+   *            {@code "jdbc:arrow-flight://"}
+   *        </td>
+   *    </tr>
+   *    <tr>
+   *        <td>1</td>
+   *        <td>IPv4 host name</td>
+   *        <td>
+   *            first word after previous group and before "{@code :}"
+   *        </td>
+   *    </tr>
+   *    <tr>
+   *        <td>2</td>
+   *        <td>IPv4 port number</td>
+   *        <td>
+   *            first number after previous group and before "{@code /?}"
+   *        </td>
+   *    </tr>
+   *    <tr>
+   *        <td>3</td>
+   *        <td>custom call parameters</td>
+   *        <td>
+   *            all parameters provided after "{@code /?}" — must follow the
+   *            pattern: "{@code key=value}" with "{@code &}" separating a
+   *            parameter from another
+   *        </td>
+   *    </tr>
+   * </table>
+   *
+   * @param url The url to parse.
+   * @return the parsed arguments.
+   * @throws SQLException If an error occurs while trying to parse the URL.
+   */
+  private Map<Object, Object> getUrlsArgs(String url)
+      throws SQLException {
+
+    /*
+     *
+     * Perhaps this logic should be inside a utility class, separated from this
+     * one, so as to better delegate responsibilities and concerns throughout
+     * the code and increase maintainability.
+     *
+     * =====
+     *
+     * Keep in mind that the URL must ALWAYS follow the pattern:
+     * "jdbc:arrow-flight://<host>:<port>[/?param1=value1&param2=value2&(...)]."
+     *
+     */
+
+    if (!url.startsWith("jdbc:")) {
+      throw new SQLException("Malformed/invalid URL!");

Review comment:
       Can we put the reason? ("Must start with jdbc:")

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcDriver.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.driver.jdbc;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.org.apache.http.NameValuePair;
+import org.apache.calcite.avatica.org.apache.http.client.utils.URLEncodedUtils;
+
+/**
+ * JDBC driver for querying data from an Apache Arrow Flight server.
+ */
+public class ArrowFlightJdbcDriver extends UnregisteredDriver {
+
+  private static final String CONNECT_STRING_PREFIX = "jdbc:arrow-flight://";
+  private static DriverVersion version;
+
+  static {
+    // Special code for supporting Java9 and higher.
+    // Netty requires some extra properties to unlock some native memory management api
+    // Setting this property if not already set externally
+    // This has to be done before any netty class is being loaded
+    final String key = "cfjd.io.netty.tryReflectionSetAccessible";
+    final String tryReflectionSetAccessible = System.getProperty(key);
+    if (tryReflectionSetAccessible == null) {
+      System.setProperty(key, Boolean.TRUE.toString());
+    }
+
+    new ArrowFlightJdbcDriver().register();
+  }
+
+  @Override
+  public ArrowFlightConnection connect(final String url, final Properties info)
+      throws SQLException {
+    final Properties properties = new Properties(info);
+    properties.putAll(info);
+
+    if (url != null) {
+      final Map<Object, Object> propertiesFromUrl = getUrlsArgs(url);
+      properties.putAll(propertiesFromUrl);
+    }
+
+    try {
+      return ArrowFlightConnection.createNewConnection(
+          this,
+          factory,
+          url,
+          properties,
+          new RootAllocator(Long.MAX_VALUE));
+    } catch (AssertionError | FlightRuntimeException e) {
+      throw new SQLException("Failed to connect.", e);
+    }
+  }
+
+  @Override
+  protected String getFactoryClassName(final JdbcVersion jdbcVersion) {
+    return ArrowFlightJdbcFactory.class.getName();
+  }
+
+  @Override
+  protected DriverVersion createDriverVersion() {
+
+    CreateVersionIfNull:
+    {
+
+      if (version != null) {
+        break CreateVersionIfNull;

Review comment:
       Why the use of a break? (Is this even valid outside of a loop?)

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcDriver.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.driver.jdbc;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.org.apache.http.NameValuePair;
+import org.apache.calcite.avatica.org.apache.http.client.utils.URLEncodedUtils;
+
+/**
+ * JDBC driver for querying data from an Apache Arrow Flight server.
+ */
+public class ArrowFlightJdbcDriver extends UnregisteredDriver {
+
+  private static final String CONNECT_STRING_PREFIX = "jdbc:arrow-flight://";
+  private static DriverVersion version;
+
+  static {
+    // Special code for supporting Java9 and higher.
+    // Netty requires some extra properties to unlock some native memory management api
+    // Setting this property if not already set externally
+    // This has to be done before any netty class is being loaded
+    final String key = "cfjd.io.netty.tryReflectionSetAccessible";
+    final String tryReflectionSetAccessible = System.getProperty(key);
+    if (tryReflectionSetAccessible == null) {
+      System.setProperty(key, Boolean.TRUE.toString());
+    }
+
+    new ArrowFlightJdbcDriver().register();
+  }
+
+  @Override
+  public ArrowFlightConnection connect(final String url, final Properties info)
+      throws SQLException {
+    final Properties properties = new Properties(info);
+    properties.putAll(info);
+
+    if (url != null) {
+      final Map<Object, Object> propertiesFromUrl = getUrlsArgs(url);
+      properties.putAll(propertiesFromUrl);
+    }
+
+    try {
+      return ArrowFlightConnection.createNewConnection(
+          this,
+          factory,
+          url,
+          properties,
+          new RootAllocator(Long.MAX_VALUE));
+    } catch (AssertionError | FlightRuntimeException e) {
+      throw new SQLException("Failed to connect.", e);
+    }
+  }
+
+  @Override
+  protected String getFactoryClassName(final JdbcVersion jdbcVersion) {
+    return ArrowFlightJdbcFactory.class.getName();
+  }
+
+  @Override
+  protected DriverVersion createDriverVersion() {
+
+    CreateVersionIfNull:
+    {
+
+      if (version != null) {
+        break CreateVersionIfNull;
+      }
+
+      try (Reader reader = new BufferedReader(new InputStreamReader(
+          this.getClass().getResourceAsStream("/properties/flight.properties"),
+          StandardCharsets.UTF_8))) {
+        final Properties properties = new Properties();
+        properties.load(reader);
+
+        final String parentName = properties
+            .getProperty("org.apache.arrow.flight.name");
+        final String parentVersion = properties
+            .getProperty("org.apache.arrow.flight.version");
+        final String[] pVersion = parentVersion.split("\\.");
+
+        final int parentMajorVersion = Integer.parseInt(pVersion[0]);
+        final int parentMinorVersion = Integer.parseInt(pVersion[1]);
+
+        final String childName = properties
+            .getProperty("org.apache.arrow.flight.jdbc-driver.name");
+        final String childVersion = properties
+            .getProperty("org.apache.arrow.flight.jdbc-driver.version");
+        final String[] cVersion = childVersion.split("\\.");
+
+        final int childMajorVersion = Integer.parseInt(cVersion[0]);
+        final int childMinorVersion = Integer.parseInt(cVersion[1]);
+
+        version = new DriverVersion(childName, childVersion, parentName,
+            parentVersion, true, childMajorVersion, childMinorVersion,
+            parentMajorVersion, parentMinorVersion);
+      } catch (final IOException e) {
+        throw new RuntimeException("Failed to load driver version.", e);
+      }
+    }
+
+    return version;
+  }
+
+  @Override
+  public Meta createMeta(final AvaticaConnection connection) {
+    return new ArrowFlightMetaImpl((ArrowFlightConnection) connection);
+  }
+
+  @Override
+  protected String getConnectStringPrefix() {
+    return CONNECT_STRING_PREFIX;
+  }
+
+  @Override
+  public boolean acceptsURL(final String url) throws SQLException {
+    return Preconditions.checkNotNull(url).startsWith(CONNECT_STRING_PREFIX);
+  }
+
+  /**
+   * Parses the provided url based on the format this driver accepts, retrieving
+   * arguments after the {@link #CONNECT_STRING_PREFIX}.
+   * <p>
+   * This method gets the args if the provided URL follows this pattern:
+   * {@code jdbc:arrow-flight://<host>:<port>[/?key1=val1&key2=val2&(...)]}
+   *
+   * <table border="1">
+   *    <tr>
+   *        <td>Group</td>
+   *        <td>Definition</td>
+   *        <td>Value</td>
+   *    </tr>
+   *    <tr>
+   *        <td>? — inaccessible</td>
+   *        <td>{@link #getConnectStringPrefix}</td>
+   *        <td>
+   *            the URL prefix accepted by this driver, i.e.,
+   *            {@code "jdbc:arrow-flight://"}
+   *        </td>
+   *    </tr>
+   *    <tr>
+   *        <td>1</td>
+   *        <td>IPv4 host name</td>
+   *        <td>
+   *            first word after previous group and before "{@code :}"
+   *        </td>
+   *    </tr>
+   *    <tr>
+   *        <td>2</td>
+   *        <td>IPv4 port number</td>
+   *        <td>
+   *            first number after previous group and before "{@code /?}"
+   *        </td>
+   *    </tr>
+   *    <tr>
+   *        <td>3</td>
+   *        <td>custom call parameters</td>
+   *        <td>
+   *            all parameters provided after "{@code /?}" — must follow the
+   *            pattern: "{@code key=value}" with "{@code &}" separating a
+   *            parameter from another
+   *        </td>
+   *    </tr>
+   * </table>
+   *
+   * @param url The url to parse.
+   * @return the parsed arguments.
+   * @throws SQLException If an error occurs while trying to parse the URL.
+   */
+  private Map<Object, Object> getUrlsArgs(String url)
+      throws SQLException {
+
+    /*
+     *
+     * Perhaps this logic should be inside a utility class, separated from this
+     * one, so as to better delegate responsibilities and concerns throughout
+     * the code and increase maintainability.
+     *
+     * =====
+     *
+     * Keep in mind that the URL must ALWAYS follow the pattern:
+     * "jdbc:arrow-flight://<host>:<port>[/?param1=value1&param2=value2&(...)]."
+     *
+     */
+
+    if (!url.startsWith("jdbc:")) {
+      throw new SQLException("Malformed/invalid URL!");
+    }
+
+    // It's necessary to use a string without "jdbc:" at the beginning to be parsed as a valid URL.
+    url = url.substring(5);
+
+    final URI uri;
+
+    try {
+      uri = URI.create(url);
+    } catch (final IllegalArgumentException e) {
+      throw new SQLException("Malformed/invalid URL!", e);
+    }
+
+    if (!Objects.equals(uri.getScheme(), "arrow-flight")) {
+      throw new SQLException("Malformed/invalid URL!");

Review comment:
       Can we put the reason? ("URL scheme must be…")

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java
##########
@@ -0,0 +1,516 @@
+/*
+ * 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.driver.jdbc.client;
+
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.arrow.driver.jdbc.client.utils.ClientAuthenticationUtils;
+import org.apache.arrow.flight.CallOption;
+import org.apache.arrow.flight.FlightClient;
+import org.apache.arrow.flight.FlightClientMiddleware;
+import org.apache.arrow.flight.FlightEndpoint;
+import org.apache.arrow.flight.FlightInfo;
+import org.apache.arrow.flight.FlightStream;
+import org.apache.arrow.flight.Location;
+import org.apache.arrow.flight.auth2.ClientBearerHeaderHandler;
+import org.apache.arrow.flight.auth2.ClientIncomingAuthHeaderMiddleware;
+import org.apache.arrow.flight.sql.FlightSqlClient;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlInfo;
+import org.apache.arrow.flight.sql.util.TableRef;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.util.AutoCloseables;
+import org.apache.arrow.util.Preconditions;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.calcite.avatica.Meta.StatementType;
+
+/**
+ * A {@link FlightSqlClient} handler.
+ */
+public final class ArrowFlightSqlClientHandler implements AutoCloseable {
+
+  private final FlightSqlClient sqlClient;
+  private final Set<CallOption> options = new HashSet<>();
+
+  ArrowFlightSqlClientHandler(final FlightSqlClient sqlClient,
+                              final Collection<CallOption> options) {
+    this.options.addAll(options);
+    this.sqlClient = Preconditions.checkNotNull(sqlClient);
+  }
+
+  /**
+   * Creates a new {@link ArrowFlightSqlClientHandler} from the provided {@code client} and {@code options}.
+   *
+   * @param client  the {@link FlightClient} to manage under a {@link FlightSqlClient} wrapper.
+   * @param options the {@link CallOption}s to persist in between subsequent client calls.
+   * @return a new {@link ArrowFlightSqlClientHandler}.
+   */
+  public static ArrowFlightSqlClientHandler createNewHandler(final FlightClient client,
+                                                             final Collection<CallOption> options) {
+    return new ArrowFlightSqlClientHandler(new FlightSqlClient(client), options);
+  }
+
+  /**
+   * Gets the {@link #options} for the subsequent calls from this handler.
+   *
+   * @return the {@link CallOption}s.
+   */
+  private CallOption[] getOptions() {
+    return options.toArray(new CallOption[0]);
+  }
+
+  /**
+   * Makes an RPC "getStream" request based on the provided {@link FlightInfo}
+   * object. Retrieves the result of the query previously prepared with "getInfo."
+   *
+   * @param flightInfo The {@link FlightInfo} instance from which to fetch results.
+   * @return a {@code FlightStream} of results.
+   */
+  public List<FlightStream> getStreams(final FlightInfo flightInfo) {
+    return flightInfo.getEndpoints().stream()
+        .map(FlightEndpoint::getTicket)
+        .map(ticket -> sqlClient.getStream(ticket, getOptions()))
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * Makes an RPC "getInfo" request based on the provided {@code query}
+   * object.
+   *
+   * @param query The query.
+   * @return a {@code FlightStream} of results.
+   */
+  public FlightInfo getInfo(final String query) {
+    return sqlClient.execute(query, getOptions());
+  }
+
+  @Override
+  public void close() throws SQLException {
+    try {
+      AutoCloseables.close(sqlClient);
+    } catch (final Exception e) {
+      throw new SQLException("Failed to clean up client resources.", e);
+    }
+  }
+
+  /**
+   * A prepared statement handler.
+   */
+  public interface PreparedStatement extends AutoCloseable {
+    /**
+     * Executes this {@link PreparedStatement}.
+     *
+     * @return the {@link FlightInfo} representing the outcome of this query execution.
+     * @throws SQLException on error.
+     */
+    FlightInfo executeQuery() throws SQLException;
+
+    /**
+     * Executes a {@link StatementType#UPDATE} query.
+     *
+     * @return the number of rows affected.
+     */
+    long executeUpdate();
+
+    /**
+     * Gets the {@link StatementType} of this {@link PreparedStatement}.
+     *
+     * @return the Statement Type.
+     */
+    StatementType getType();
+
+    /**
+     * Gets the {@link Schema} of this {@link PreparedStatement}.
+     *
+     * @return {@link Schema}.
+     */
+    Schema getDataSetSchema();
+
+    @Override
+    void close();
+  }
+
+  /**
+   * Creates a new {@link PreparedStatement} for the given {@code query}.
+   *
+   * @param query the SQL query.
+   * @return a new prepared statement.
+   */
+  public PreparedStatement prepare(final String query) {
+    final FlightSqlClient.PreparedStatement preparedStatement =
+        sqlClient.prepare(query, getOptions());
+    return new PreparedStatement() {
+      @Override
+      public FlightInfo executeQuery() throws SQLException {
+        return preparedStatement.execute(getOptions());

Review comment:
       We should be catching and wrapping FlightRuntimeException here right?

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/accessor/impl/numeric/ArrowFlightJdbcBitVectorAccessor.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.driver.jdbc.accessor.impl.numeric;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.function.IntSupplier;
+
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessor;
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessorFactory;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.holders.NullableBitHolder;
+
+/**
+ * Accessor for the arrow {@link BitVector}.
+ */
+public class ArrowFlightJdbcBitVectorAccessor extends ArrowFlightJdbcAccessor {
+
+  private final BitVector vector;
+  private final NullableBitHolder holder;
+  private static final int BYTES_T0_ALLOCATE = 1;
+
+  /**
+   * Constructor for the BitVectorAccessor.
+   *
+   * @param vector             an instance of a {@link BitVector}.
+   * @param currentRowSupplier a supplier to check which row is being accessed.
+   * @param setCursorWasNull   the consumer to set if value was null.
+   */
+  public ArrowFlightJdbcBitVectorAccessor(BitVector vector, IntSupplier currentRowSupplier,
+                                          ArrowFlightJdbcAccessorFactory.WasNullConsumer setCursorWasNull) {
+    super(currentRowSupplier, setCursorWasNull);
+    this.vector = vector;
+    this.holder = new NullableBitHolder();
+  }
+
+  @Override
+  public Class<?> getObjectClass() {
+    return Long.class;

Review comment:
       Not Boolean.class?

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcPooledConnection.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.driver.jdbc;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+import javax.sql.ConnectionEvent;
+import javax.sql.ConnectionEventListener;
+import javax.sql.PooledConnection;
+import javax.sql.StatementEventListener;
+
+import org.apache.arrow.driver.jdbc.utils.ConnectionWrapper;
+
+/**
+ * {@link PooledConnection} implementation for Arrow Flight JDBC Driver.
+ */
+public class ArrowFlightJdbcPooledConnection implements PooledConnection {
+
+  private final ArrowFlightConnection connection;
+  private final Set<ConnectionEventListener> eventListeners;
+  private final Set<StatementEventListener> statementEventListeners;
+
+  private final class ConnectionHandle extends ConnectionWrapper {

Review comment:
       nit: it might be a little clearer to make this a static class and explicitly pass through the connection.

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/accessor/impl/calendar/ArrowFlightJdbcTimeVectorAccessor.java
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.driver.jdbc.accessor.impl.calendar;
+
+import static org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeVectorGetter.Getter;
+import static org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeVectorGetter.Holder;
+import static org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeVectorGetter.createGetter;
+
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Calendar;
+import java.util.concurrent.TimeUnit;
+import java.util.function.IntSupplier;
+
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessor;
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessorFactory;
+import org.apache.arrow.driver.jdbc.utils.DateTimeUtils;
+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.ValueVector;
+
+/**
+ * Accessor for the Arrow types: {@link TimeNanoVector}, {@link TimeMicroVector}, {@link TimeMilliVector}
+ * and {@link TimeSecVector}.
+ */
+public class ArrowFlightJdbcTimeVectorAccessor extends ArrowFlightJdbcAccessor {
+
+  private final Getter getter;
+  private final TimeUnit timeUnit;
+  private final Holder holder;
+
+  /**
+   * Instantiate an accessor for a {@link TimeNanoVector}.
+   *
+   * @param vector             an instance of a TimeNanoVector.
+   * @param currentRowSupplier the supplier to track the lines.
+   * @param setCursorWasNull   the consumer to set if value was null.
+   */
+  public ArrowFlightJdbcTimeVectorAccessor(TimeNanoVector vector, IntSupplier currentRowSupplier,
+                                           ArrowFlightJdbcAccessorFactory.WasNullConsumer setCursorWasNull) {
+    super(currentRowSupplier, setCursorWasNull);
+    this.holder = new Holder();
+    this.getter = createGetter(vector);
+    this.timeUnit = getTimeUnitForVector(vector);
+  }
+
+  /**
+   * Instantiate an accessor for a {@link TimeMicroVector}.
+   *
+   * @param vector             an instance of a TimeMicroVector.
+   * @param currentRowSupplier the supplier to track the lines.
+   * @param setCursorWasNull   the consumer to set if value was null.
+   */
+  public ArrowFlightJdbcTimeVectorAccessor(TimeMicroVector vector, IntSupplier currentRowSupplier,
+                                           ArrowFlightJdbcAccessorFactory.WasNullConsumer setCursorWasNull) {
+    super(currentRowSupplier, setCursorWasNull);
+    this.holder = new Holder();
+    this.getter = createGetter(vector);
+    this.timeUnit = getTimeUnitForVector(vector);
+  }
+
+  /**
+   * Instantiate an accessor for a {@link TimeMilliVector}.
+   *
+   * @param vector             an instance of a TimeMilliVector.
+   * @param currentRowSupplier the supplier to track the lines.
+   */
+  public ArrowFlightJdbcTimeVectorAccessor(TimeMilliVector vector, IntSupplier currentRowSupplier,
+                                           ArrowFlightJdbcAccessorFactory.WasNullConsumer setCursorWasNull) {
+    super(currentRowSupplier, setCursorWasNull);
+    this.holder = new Holder();
+    this.getter = createGetter(vector);
+    this.timeUnit = getTimeUnitForVector(vector);
+  }
+
+  /**
+   * Instantiate an accessor for a {@link TimeSecVector}.
+   *
+   * @param vector             an instance of a TimeSecVector.
+   * @param currentRowSupplier the supplier to track the lines.
+   */
+  public ArrowFlightJdbcTimeVectorAccessor(TimeSecVector vector, IntSupplier currentRowSupplier,
+                                           ArrowFlightJdbcAccessorFactory.WasNullConsumer setCursorWasNull) {
+    super(currentRowSupplier, setCursorWasNull);
+    this.holder = new Holder();
+    this.getter = createGetter(vector);
+    this.timeUnit = getTimeUnitForVector(vector);
+  }
+
+  @Override
+  public Class<?> getObjectClass() {
+    return Time.class;
+  }
+
+  @Override
+  public Object getObject() {
+    return this.getTime(null);
+  }
+
+  @Override
+  public Time getTime(Calendar calendar) {
+    getter.get(getCurrentRow(), holder);
+    this.wasNull = holder.isSet == 0;
+    this.wasNullConsumer.setWasNull(this.wasNull);
+    if (this.wasNull) {
+      return null;
+    }
+
+    long value = holder.value;
+    long milliseconds = this.timeUnit.toMillis(value);
+
+    return new Time(DateTimeUtils.applyCalendarOffset(milliseconds, calendar));
+  }
+
+  @Override
+  public Timestamp getTimestamp(Calendar calendar) {
+    Time time = getTime(calendar);
+    if (time == null) {
+      return null;
+    }
+    return new Timestamp(time.getTime());

Review comment:
       I don't think we can get from Time to Timestamp.

##########
File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/utils/ConnectionWrapper.java
##########
@@ -0,0 +1,344 @@
+/*
+ * 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.driver.jdbc.utils;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.CallableStatement;
+import java.sql.Clob;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.NClob;
+import java.sql.PreparedStatement;
+import java.sql.SQLClientInfoException;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Savepoint;
+import java.sql.Statement;
+import java.sql.Struct;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Executor;
+
+import org.apache.arrow.driver.jdbc.ArrowFlightJdbcPooledConnection;
+
+/**
+ * Auxiliary wrapper class for {@link Connection}, used on {@link ArrowFlightJdbcPooledConnection}.
+ */
+public class ConnectionWrapper implements Connection {

Review comment:
       Since this appears to be a pure proxy, it might be worth noting that fact (aka this is purely to serve as a base class and factor out the boilerplate elsewhere)




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