laurentgo commented on a change in pull request #10906:
URL: https://github.com/apache/arrow/pull/10906#discussion_r695098610



##########
File path: 
java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/StatementContext.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.flight.sql.example;
+
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.Objects;
+import java.util.Optional;
+
+import org.apache.arrow.flight.sql.FlightSqlProducer;
+import org.apache.arrow.util.AutoCloseables;
+
+/**
+ * Context for {@link T} to be persisted in memory in between {@link 
FlightSqlProducer} calls.
+ *
+ * @param <T> the {@link Statement} to be persisted.
+ */
+public final class StatementContext<T extends Statement> implements 
AutoCloseable {
+
+  private final T statement;
+  private final String query;
+
+  public StatementContext(final T statement, final String query) {
+    this.statement = Objects.requireNonNull(statement, "statement cannot be 
null.");
+    this.query = query;
+  }
+
+  public StatementContext(final T statement) {
+    this(statement, null);
+  }
+
+  /**
+   * Gets the statement wrapped by this {@link StatementContext}.
+   *
+   * @return the inner statement.
+   */
+  public T getStatement() {
+    return statement;
+  }
+
+  /**
+   * Gets the optional SQL query wrapped by this {@link StatementContext}.
+   *
+   * @return the SQL query if present; empty otherwise.
+   */
+  public Optional<String> getQuery() {

Review comment:
       if this method returns an optional, why the constructor doesn't accept 
an optional too?

##########
File path: java/flight/flight-sql/pom.xml
##########
@@ -0,0 +1,243 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>arrow-flight</artifactId>
+    <groupId>org.apache.arrow</groupId>
+    <version>6.0.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>flight-sql</artifactId>
+  <name>Arrow Flight SQL</name>
+  <description>(Experimental)Contains utility classes to expose Flight SQL 
semantics for clients and servers over Arrow Flight</description>
+  <packaging>jar</packaging>
+
+  <properties>
+    <dep.grpc.version>1.30.2</dep.grpc.version>

Review comment:
       I guess those are the same versions used by `flight-grpc` module? Maybe 
those should be in some common location to make sure they stay in sync

##########
File path: java/flight/flight-sql/pom.xml
##########
@@ -0,0 +1,243 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>arrow-flight</artifactId>
+    <groupId>org.apache.arrow</groupId>
+    <version>6.0.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>flight-sql</artifactId>
+  <name>Arrow Flight SQL</name>
+  <description>(Experimental)Contains utility classes to expose Flight SQL 
semantics for clients and servers over Arrow Flight</description>
+  <packaging>jar</packaging>
+
+  <properties>
+    <dep.grpc.version>1.30.2</dep.grpc.version>
+    <dep.protobuf.version>3.7.1</dep.protobuf.version>
+    <forkCount>1</forkCount>
+    <!-- Overridden at runtime! -->
+    <server.port>0000</server.port>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>flight-core</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-unix-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-kqueue</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-epoll</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-format</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-netty</artifactId>
+      <version>${project.version}</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-jdbc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>${dep.protobuf.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-api</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-vector</artifactId>
+      <version>${project.version}</version>
+      <classifier>${arrow.vector.classifier}</classifier>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.derby</groupId>
+      <artifactId>derby</artifactId>
+      <version>10.14.2.0</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-dbcp2</artifactId>
+      <version>2.7.0</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-pool2</artifactId>
+      <version>2.8.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <extensions>

Review comment:
       since the extension is used by all flight modules + gandiva, maybe a 
good idea to move it to a parent pom

##########
File path: 
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java
##########
@@ -55,15 +56,73 @@
  */
 public final class JdbcToArrowConfig {
 
+  private static final BiFunction<JdbcFieldInfo, Calendar, ArrowType> 
DEFAULT_JDBC_TO_ARROW_TYPE_CONVERTER =

Review comment:
       Is it better to make this an unnamed lambda function returned by a 
public static method vs having a static function taking a jdbc info type + a 
calendar and returning an arrow type? The latter seems cleaner (and because it 
is a public method, it might help for testing?) and Java is quite efficient at 
adapting a method to a function...
   
   

##########
File path: 
java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java
##########
@@ -0,0 +1,504 @@
+/*
+ * 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.flight.sql;
+
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.ActionClosePreparedStatementRequest;
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementRequest;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetCatalogs;
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.CommandGetExportedKeys;
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.CommandGetImportedKeys;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetPrimaryKeys;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSchemas;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSqlInfo;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTableTypes;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTables;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementQuery;
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementUpdate;
+import static org.apache.arrow.flight.sql.impl.FlightSql.DoPutUpdateResult;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.sql.SQLException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicLong;
+
+import javax.annotation.Nullable;
+
+import org.apache.arrow.flight.Action;
+import org.apache.arrow.flight.CallOption;
+import org.apache.arrow.flight.FlightClient;
+import org.apache.arrow.flight.FlightDescriptor;
+import org.apache.arrow.flight.FlightInfo;
+import org.apache.arrow.flight.FlightStream;
+import org.apache.arrow.flight.PutResult;
+import org.apache.arrow.flight.Result;
+import org.apache.arrow.flight.SchemaResult;
+import org.apache.arrow.flight.SyncPutListener;
+import org.apache.arrow.flight.Ticket;
+import org.apache.arrow.flight.sql.impl.FlightSql;
+import 
org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementResult;
+import 
org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery;
+import org.apache.arrow.memory.ArrowBuf;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.pojo.Schema;
+
+import com.google.protobuf.Any;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.StringValue;
+
+import io.grpc.Status;
+
+/**
+ * Flight client with Flight SQL semantics.
+ */
+public class FlightSqlClient {
+  private FlightClient client;
+
+  public FlightSqlClient(FlightClient client) {
+    this.client = client;
+  }
+
+  /**
+   * Execute a query on the server.
+   *
+   * @param query The query to execute.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo execute(String query) {
+    final CommandStatementQuery.Builder builder = 
CommandStatementQuery.newBuilder();
+    builder.setQuery(query);
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor);
+  }
+
+  /**
+   * Execute an update query on the server.
+   *
+   * @param query The query to execute.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public long executeUpdate(String query) {
+    final CommandStatementUpdate.Builder builder = 
CommandStatementUpdate.newBuilder();
+    builder.setQuery(query);
+
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    final SyncPutListener putListener = new SyncPutListener();
+    client.startPut(descriptor, VectorSchemaRoot.of(), putListener);
+
+    try {
+      final PutResult read = putListener.read();
+      try (final ArrowBuf metadata = read.getApplicationMetadata()) {
+        final DoPutUpdateResult doPutUpdateResult = 
DoPutUpdateResult.parseFrom(metadata.nioBuffer());
+        return doPutUpdateResult.getRecordCount();
+      }
+    } catch (InterruptedException | ExecutionException | 
InvalidProtocolBufferException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Request a list of catalogs.
+   *
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getCatalogs() {
+    final CommandGetCatalogs.Builder builder = CommandGetCatalogs.newBuilder();
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor);
+  }
+
+  /**
+   * Request a list of schemas.
+   *
+   * @param catalog             The catalog.
+   * @param schemaFilterPattern The schema filter pattern.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getSchemas(final String catalog, final String 
schemaFilterPattern) {
+    final CommandGetSchemas.Builder builder = CommandGetSchemas.newBuilder();
+
+    if (catalog != null) {
+      builder.setCatalog(StringValue.newBuilder().setValue(catalog).build());
+    }
+
+    if (schemaFilterPattern != null) {
+      
builder.setSchemaFilterPattern(StringValue.newBuilder().setValue(schemaFilterPattern).build());
+    }
+
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor);
+  }
+
+  /**
+   * Get schema for a stream.
+   *
+   * @param descriptor The descriptor for the stream.
+   * @param options    RPC-layer hints for this call.
+   */
+  public SchemaResult getSchema(FlightDescriptor descriptor, CallOption... 
options) {
+    return this.client.getSchema(descriptor, options);
+  }
+
+  /**
+   * Retrieve a stream from the server.
+   *
+   * @param ticket  The ticket granting access to the data stream.
+   * @param options RPC-layer hints for this call.
+   */
+  public FlightStream getStream(Ticket ticket, CallOption... options) {
+    return this.client.getStream(ticket, options);
+  }
+
+  /**
+   * Request a set of Flight SQL metadata.
+   *
+   * @param info The set of metadata to retrieve. None to retrieve all 
metadata.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getSqlInfo(final @Nullable int... info) {
+    final CommandGetSqlInfo.Builder builder = CommandGetSqlInfo.newBuilder();
+    for (final int pieceOfInfo : Objects.isNull(info) ? new int[0] : info) {
+      builder.addInfo(pieceOfInfo);
+    }
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor);
+  }
+
+  /**
+   * Request a list of tables.
+   *
+   * @param catalog             The catalog.
+   * @param schemaFilterPattern The schema filter pattern.
+   * @param tableFilterPattern  The table filter pattern.
+   * @param tableTypes          The table types to include.
+   * @param includeSchema       True to include the schema upon return, false 
to not include the schema.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getTables(final @Nullable String catalog, final @Nullable 
String schemaFilterPattern,
+                              final @Nullable String tableFilterPattern, final 
List<String> tableTypes,
+                              final boolean includeSchema) {
+    final CommandGetTables.Builder builder = CommandGetTables.newBuilder();
+
+    if (catalog != null) {
+      builder.setCatalog(StringValue.newBuilder().setValue(catalog).build());
+    }
+
+    if (schemaFilterPattern != null) {
+      
builder.setSchemaFilterPattern(StringValue.newBuilder().setValue(schemaFilterPattern).build());
+    }
+
+    if (tableFilterPattern != null) {
+      
builder.setTableNameFilterPattern(StringValue.newBuilder().setValue(tableFilterPattern).build());
+    }
+
+    if (tableTypes != null) {
+      builder.addAllTableTypes(tableTypes);
+    }
+    builder.setIncludeSchema(includeSchema);
+
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor);
+  }
+
+  /**
+   * Request the primary keys for a table.
+   *
+   * @param catalog The catalog.
+   * @param schema  The schema.
+   * @param table   The table.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getPrimaryKeys(final @Nullable String catalog, final 
@Nullable String schema,
+                                   final @Nullable String table) {
+    final CommandGetPrimaryKeys.Builder builder = 
CommandGetPrimaryKeys.newBuilder();
+
+    if (catalog != null) {
+      builder.setCatalog(StringValue.newBuilder().setValue(catalog).build());
+    }
+
+    if (schema != null) {
+      builder.setSchema(StringValue.newBuilder().setValue(schema).build());
+    }
+
+    if (table != null) {
+      builder.setTable(StringValue.newBuilder().setValue(table).build());
+    }
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor);
+  }
+
+  /**
+   * Request to get info about keys on a table. The table, which exports the 
foreign keys, parameter must be specified.
+   *
+   * @param catalog The foreign key table catalog.
+   * @param schema  The foreign key table schema.
+   * @param table   The foreign key table.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getExportedKeys(String catalog, String schema, String 
table) {
+    if (null == table) {
+      throw Status.INVALID_ARGUMENT.asRuntimeException();
+    }
+
+    final CommandGetExportedKeys.Builder builder = 
CommandGetExportedKeys.newBuilder();
+
+    if (catalog != null) {
+      builder.setCatalog(StringValue.newBuilder().setValue(catalog).build());
+    }
+
+    if (schema != null) {
+      builder.setSchema(StringValue.newBuilder().setValue(schema).build());
+    }
+
+    builder.setTable(table).build();
+
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor);
+  }
+
+  /**
+   * Request to get info about keys on a table. The table, which imports the 
foreign keys, parameter must be specified.
+   *
+   * @param catalog The primary key table catalog.
+   * @param schema  The primary key table schema.
+   * @param table   The primary key table.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getImportedKeys(String catalog, String schema, String 
table) {
+    if (null == table) {
+      throw Status.INVALID_ARGUMENT.asRuntimeException();
+    }
+
+    final CommandGetImportedKeys.Builder builder = 
CommandGetImportedKeys.newBuilder();
+
+    if (catalog != null) {
+      builder.setCatalog(StringValue.newBuilder().setValue(catalog).build());
+    }
+
+    if (schema != null) {
+      builder.setSchema(StringValue.newBuilder().setValue(schema).build());
+    }
+
+    builder.setTable(table).build();
+
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor);
+  }
+
+  /**
+   * Request a list of table types.
+   *
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getTableTypes() {
+    final CommandGetTableTypes.Builder builder = 
CommandGetTableTypes.newBuilder();
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor);
+  }
+
+  /**
+   * Create a prepared statement on the server.
+   *
+   * @param query The query to prepare.
+   * @return The representation of the prepared statement which exists on the 
server.
+   */
+  public PreparedStatement prepare(String query) {
+    return new PreparedStatement(client, query);
+  }
+
+  /**
+   * Helper class to encapsulate Flight SQL prepared statement logic.
+   */
+  public static class PreparedStatement implements Closeable {
+    private final FlightClient client;
+    private final ActionCreatePreparedStatementResult preparedStatementResult;
+    private AtomicLong invocationCount;
+    private boolean isClosed;
+    private Schema resultSetSchema = null;
+    private Schema parameterSchema = null;
+    private VectorSchemaRoot parameterBindingRoot;
+
+    /**
+     * Constructor.
+     *
+     * @param client The client. FlightSqlPreparedStatement does not maintain 
this resource.
+     * @param sql    The query.
+     */
+    public PreparedStatement(FlightClient client, String sql) {
+      this.client = client;
+
+      final Iterator<Result> preparedStatementResults = client.doAction(new 
Action(
+          FlightSqlUtils.FLIGHT_SQL_CREATEPREPAREDSTATEMENT.getType(),
+          Any.pack(ActionCreatePreparedStatementRequest
+              .newBuilder()
+              .setQuery(sql)
+              .build())
+              .toByteArray()));
+
+      preparedStatementResult = FlightSqlUtils.unpackAndParseOrThrow(
+          preparedStatementResults.next().getBody(),
+          ActionCreatePreparedStatementResult.class);
+
+      invocationCount = new AtomicLong(0);
+      isClosed = false;
+    }
+
+    /**
+     * Set the {@link VectorSchemaRoot} containing the parameter binding from 
a preparedStatemnt
+     * operation.
+     *
+     * @param parameterBindingRoot  a {@link VectorSchemaRoot} object contain 
the values to be used in the
+     *                              PreparedStatement setters.
+     */
+    public void setParameters(VectorSchemaRoot parameterBindingRoot) {
+      this.parameterBindingRoot = parameterBindingRoot;
+    }
+
+    /**
+     * Empty the {@link VectorSchemaRoot} that contains the parameter binding 
from a preparedStatemnt
+     * operation.
+     *
+     */
+    public void clearParameters() {
+      this.parameterBindingRoot = null;
+    }
+
+    /**
+     * Returns the Schema of the resultset.
+     *
+     * @return the Schema of the resultset.
+     */
+    public Schema getResultSetSchema() {
+      if (resultSetSchema == null && 
preparedStatementResult.getDatasetSchema() != null) {
+        resultSetSchema = 
Schema.deserialize(preparedStatementResult.getDatasetSchema().asReadOnlyByteBuffer());
+      }
+      return resultSetSchema;
+    }
+
+    /**
+     * Returns the Schema of the parameters.
+     *
+     * @return the Schema of the parameters.
+     */
+    public Schema getParameterSchema() {
+      if (parameterSchema == null && 
preparedStatementResult.getParameterSchema() != null) {
+        parameterSchema = 
Schema.deserialize(preparedStatementResult.getParameterSchema().asReadOnlyByteBuffer());
+      }
+      return parameterSchema;
+    }
+
+    /**
+     * Executes the prepared statement query on the server.
+     *
+     * @return a FlightInfo object representing the stream(s) to fetch.
+     * @throws IOException if the PreparedStatement is closed.
+     */
+    public FlightInfo execute() throws IOException {
+      if (isClosed) {
+        throw new IllegalStateException("Prepared statement has already been 
closed on the server.");
+      }
+
+      final FlightDescriptor descriptor = FlightDescriptor
+          .command(Any.pack(CommandPreparedStatementQuery.newBuilder()
+              .setClientExecutionHandle(
+                  
ByteString.copyFrom(ByteBuffer.allocate(Long.BYTES).putLong(invocationCount.getAndIncrement())))
+              
.setPreparedStatementHandle(preparedStatementResult.getPreparedStatementHandle())
+              .build())
+              .toByteArray());
+
+      if (parameterBindingRoot != null) {
+        final SyncPutListener putListener = new SyncPutListener();
+
+        FlightClient.ClientStreamListener listener =
+            client.startPut(descriptor, this.parameterBindingRoot, 
putListener);
+
+        listener.putNext();
+        listener.completed();
+      }

Review comment:
       I guess it also means that the same prepared statement cannot be used 
concurrently with multiple set of parameters?

##########
File path: java/flight/flight-sql/pom.xml
##########
@@ -0,0 +1,243 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>arrow-flight</artifactId>
+    <groupId>org.apache.arrow</groupId>
+    <version>6.0.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>flight-sql</artifactId>
+  <name>Arrow Flight SQL</name>
+  <description>(Experimental)Contains utility classes to expose Flight SQL 
semantics for clients and servers over Arrow Flight</description>
+  <packaging>jar</packaging>
+
+  <properties>
+    <dep.grpc.version>1.30.2</dep.grpc.version>
+    <dep.protobuf.version>3.7.1</dep.protobuf.version>
+    <forkCount>1</forkCount>
+    <!-- Overridden at runtime! -->
+    <server.port>0000</server.port>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>flight-core</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-unix-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-kqueue</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-epoll</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-format</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-netty</artifactId>
+      <version>${project.version}</version>
+      <scope>runtime</scope>

Review comment:
       runtime or test?

##########
File path: 
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowUtils.java
##########
@@ -106,6 +107,46 @@ public static Schema jdbcToArrowSchema(ResultSetMetaData 
rsmd, Calendar calendar
     return jdbcToArrowSchema(rsmd, new JdbcToArrowConfig(new RootAllocator(0), 
calendar));
   }
 
+  /**
+   * Create Arrow {@link Schema} object for the given JDBC {@link 
ResultSetMetaData}.
+   *
+   * @param parameterMetaData The ResultSetMetaData containing the results, to 
read the JDBC metadata from.
+   * @param calendar          The calendar to use the time zone field of, to 
construct Timestamp fields from.
+   * @return {@link Schema}
+   * @throws SQLException on error
+   */
+  public static Schema jdbcToArrowSchema(final ParameterMetaData 
parameterMetaData, final Calendar calendar)
+      throws SQLException {
+    Preconditions.checkNotNull(calendar, "Calendar object can't be null");
+    Preconditions.checkNotNull(parameterMetaData);
+    final List<Field> parameterFields = new ArrayList<>();

Review comment:
       (micro-optimization): since the number of parameters is know in advance, 
it might be worth specifying the list capacity to avoid unnecessary reallocation

##########
File path: 
java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/StatementContext.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.flight.sql.example;
+
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.Objects;
+import java.util.Optional;
+
+import org.apache.arrow.flight.sql.FlightSqlProducer;
+import org.apache.arrow.util.AutoCloseables;
+
+/**
+ * Context for {@link T} to be persisted in memory in between {@link 
FlightSqlProducer} calls.
+ *
+ * @param <T> the {@link Statement} to be persisted.
+ */
+public final class StatementContext<T extends Statement> implements 
AutoCloseable {
+
+  private final T statement;
+  private final String query;
+
+  public StatementContext(final T statement, final String query) {
+    this.statement = Objects.requireNonNull(statement, "statement cannot be 
null.");
+    this.query = query;
+  }
+
+  public StatementContext(final T statement) {
+    this(statement, null);
+  }
+
+  /**
+   * Gets the statement wrapped by this {@link StatementContext}.
+   *
+   * @return the inner statement.
+   */
+  public T getStatement() {
+    return statement;
+  }
+
+  /**
+   * Gets the optional SQL query wrapped by this {@link StatementContext}.
+   *
+   * @return the SQL query if present; empty otherwise.
+   */
+  public Optional<String> getQuery() {
+    return Optional.ofNullable(query);
+  }
+
+  @Override
+  public void close() throws Exception {
+    Connection connection = statement.getConnection();
+    AutoCloseables.close(statement, connection);
+  }
+
+  @Override
+  public boolean equals(final Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (!(other instanceof StatementContext)) {
+      return false;
+    }
+    final StatementContext<?> that = (StatementContext<?>) other;
+    return getStatement().equals(that.getStatement());

Review comment:
       If the class is final (which is good), there's no need to call the 
method and the field can directly be used...

##########
File path: 
java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java
##########
@@ -148,64 +209,16 @@
 
     // set up type converter
     this.jdbcToArrowTypeConverter = jdbcToArrowTypeConverter != null ? 
jdbcToArrowTypeConverter :
-        fieldInfo -> {
-          final String timezone;
-          if (calendar != null) {
-            timezone = calendar.getTimeZone().getID();
-          } else {
-            timezone = null;
-          }
+        jdbcFieldInfo -> 
getDefaultJdbcToArrowTypeConverter().apply(jdbcFieldInfo, calendar);
+  }
 
-          switch (fieldInfo.getJdbcType()) {
-            case Types.BOOLEAN:
-            case Types.BIT:
-              return new ArrowType.Bool();
-            case Types.TINYINT:
-              return new ArrowType.Int(8, true);
-            case Types.SMALLINT:
-              return new ArrowType.Int(16, true);
-            case Types.INTEGER:
-              return new ArrowType.Int(32, true);
-            case Types.BIGINT:
-              return new ArrowType.Int(64, true);
-            case Types.NUMERIC:
-            case Types.DECIMAL:
-              int precision = fieldInfo.getPrecision();
-              int scale = fieldInfo.getScale();
-              return new ArrowType.Decimal(precision, scale, 128);
-            case Types.REAL:
-            case Types.FLOAT:
-              return new ArrowType.FloatingPoint(SINGLE);
-            case Types.DOUBLE:
-              return new ArrowType.FloatingPoint(DOUBLE);
-            case Types.CHAR:
-            case Types.NCHAR:
-            case Types.VARCHAR:
-            case Types.NVARCHAR:
-            case Types.LONGVARCHAR:
-            case Types.LONGNVARCHAR:
-            case Types.CLOB:
-              return new ArrowType.Utf8();
-            case Types.DATE:
-              return new ArrowType.Date(DateUnit.DAY);
-            case Types.TIME:
-              return new ArrowType.Time(TimeUnit.MILLISECOND, 32);
-            case Types.TIMESTAMP:
-              return new ArrowType.Timestamp(TimeUnit.MILLISECOND, timezone);
-            case Types.BINARY:
-            case Types.VARBINARY:
-            case Types.LONGVARBINARY:
-            case Types.BLOB:
-              return new ArrowType.Binary();
-            case Types.ARRAY:
-              return new ArrowType.List();
-            case Types.NULL:
-              return new ArrowType.Null();
-            default:
-              // no-op, shouldn't get here
-              return null;
-          }
-        };
+  /**
+   * Gets the default JDBC-type-to-Arrow-type converter.
+   *
+   * @return the default converter.
+   */
+  public static BiFunction<JdbcFieldInfo, Calendar, ArrowType> 
getDefaultJdbcToArrowTypeConverter() {

Review comment:
       See my comment for the constant, but returning a function object 
wrapping instead of the method itself is not a common pattern imho.

##########
File path: java/flight/flight-sql/pom.xml
##########
@@ -0,0 +1,243 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>arrow-flight</artifactId>
+    <groupId>org.apache.arrow</groupId>
+    <version>6.0.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>flight-sql</artifactId>
+  <name>Arrow Flight SQL</name>
+  <description>(Experimental)Contains utility classes to expose Flight SQL 
semantics for clients and servers over Arrow Flight</description>
+  <packaging>jar</packaging>
+
+  <properties>
+    <dep.grpc.version>1.30.2</dep.grpc.version>
+    <dep.protobuf.version>3.7.1</dep.protobuf.version>
+    <forkCount>1</forkCount>
+    <!-- Overridden at runtime! -->
+    <server.port>0000</server.port>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>flight-core</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-unix-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-kqueue</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-epoll</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-format</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-netty</artifactId>
+      <version>${project.version}</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-jdbc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>${dep.protobuf.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-api</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-vector</artifactId>
+      <version>${project.version}</version>
+      <classifier>${arrow.vector.classifier}</classifier>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.derby</groupId>
+      <artifactId>derby</artifactId>
+      <version>10.14.2.0</version>

Review comment:
       This version is from 2018 and the latest version is 10.15.2.0 (from Feb 
2020). Any specific reason for not using the latest version?
   (same for the other test dependencies)

##########
File path: java/flight/flight-sql/pom.xml
##########
@@ -0,0 +1,243 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>arrow-flight</artifactId>
+    <groupId>org.apache.arrow</groupId>
+    <version>6.0.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>flight-sql</artifactId>
+  <name>Arrow Flight SQL</name>
+  <description>(Experimental)Contains utility classes to expose Flight SQL 
semantics for clients and servers over Arrow Flight</description>
+  <packaging>jar</packaging>
+
+  <properties>
+    <dep.grpc.version>1.30.2</dep.grpc.version>
+    <dep.protobuf.version>3.7.1</dep.protobuf.version>
+    <forkCount>1</forkCount>
+    <!-- Overridden at runtime! -->
+    <server.port>0000</server.port>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>flight-core</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-unix-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-kqueue</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-epoll</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-format</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-netty</artifactId>
+      <version>${project.version}</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-jdbc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>${dep.protobuf.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-api</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-vector</artifactId>
+      <version>${project.version}</version>
+      <classifier>${arrow.vector.classifier}</classifier>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.derby</groupId>
+      <artifactId>derby</artifactId>
+      <version>10.14.2.0</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-dbcp2</artifactId>
+      <version>2.7.0</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-pool2</artifactId>
+      <version>2.8.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <extensions>
+      <!-- provides os.detected.classifier (i.e. linux-x86_64, osx-x86_64) 
property -->
+      <extension>
+          <groupId>kr.motd.maven</groupId>
+          <artifactId>os-maven-plugin</artifactId>
+          <version>1.5.0.Final</version>
+      </extension>
+    </extensions>
+    <plugins>
+      <plugin>
+        <groupId>org.xolstice.maven.plugins</groupId>
+        <artifactId>protobuf-maven-plugin</artifactId>
+        <version>0.5.0</version>
+        <configuration>
+          
<protocArtifact>com.google.protobuf:protoc:${dep.protobuf.version}:exe:${os.detected.classifier}</protocArtifact>
+          <pluginId>grpc-java</pluginId>
+          
<pluginArtifact>io.grpc:protoc-gen-grpc-java:${dep.grpc.version}:exe:${os.detected.classifier}</pluginArtifact>
+        </configuration>
+        <executions>
+          <execution>
+            <id>proto-compile</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <protoSourceRoot>${basedir}/../../../format/</protoSourceRoot>
+            </configuration>
+            <goals>
+              <goal>compile</goal>
+              <goal>compile-custom</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>proto-test-compile</id>
+            <phase>generate-test-sources</phase>
+            <goals>
+              <goal>test-compile</goal>
+              <goal>test-compile-custom</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>3.2.0</version>

Review comment:
       Also used by the flight-core module, may be worth adding to 
`pluginManagement`

##########
File path: java/flight/flight-sql/pom.xml
##########
@@ -0,0 +1,243 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>arrow-flight</artifactId>
+    <groupId>org.apache.arrow</groupId>
+    <version>6.0.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>flight-sql</artifactId>
+  <name>Arrow Flight SQL</name>
+  <description>(Experimental)Contains utility classes to expose Flight SQL 
semantics for clients and servers over Arrow Flight</description>
+  <packaging>jar</packaging>
+
+  <properties>
+    <dep.grpc.version>1.30.2</dep.grpc.version>
+    <dep.protobuf.version>3.7.1</dep.protobuf.version>
+    <forkCount>1</forkCount>
+    <!-- Overridden at runtime! -->
+    <server.port>0000</server.port>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>flight-core</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-unix-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-kqueue</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-epoll</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-format</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-netty</artifactId>
+      <version>${project.version}</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-jdbc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>${dep.protobuf.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-api</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-vector</artifactId>
+      <version>${project.version}</version>
+      <classifier>${arrow.vector.classifier}</classifier>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.derby</groupId>
+      <artifactId>derby</artifactId>
+      <version>10.14.2.0</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-dbcp2</artifactId>
+      <version>2.7.0</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-pool2</artifactId>
+      <version>2.8.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <extensions>
+      <!-- provides os.detected.classifier (i.e. linux-x86_64, osx-x86_64) 
property -->
+      <extension>
+          <groupId>kr.motd.maven</groupId>
+          <artifactId>os-maven-plugin</artifactId>
+          <version>1.5.0.Final</version>
+      </extension>
+    </extensions>
+    <plugins>
+      <plugin>
+        <groupId>org.xolstice.maven.plugins</groupId>
+        <artifactId>protobuf-maven-plugin</artifactId>
+        <version>0.5.0</version>

Review comment:
       You should use the latest version of the plugin which addresses several 
issues (even better, the plugin version should be probably be managed by 
pluginManagement at the top level)

##########
File path: java/flight/flight-sql/pom.xml
##########
@@ -0,0 +1,243 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>arrow-flight</artifactId>
+    <groupId>org.apache.arrow</groupId>
+    <version>6.0.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>flight-sql</artifactId>
+  <name>Arrow Flight SQL</name>
+  <description>(Experimental)Contains utility classes to expose Flight SQL 
semantics for clients and servers over Arrow Flight</description>
+  <packaging>jar</packaging>
+
+  <properties>
+    <dep.grpc.version>1.30.2</dep.grpc.version>
+    <dep.protobuf.version>3.7.1</dep.protobuf.version>
+    <forkCount>1</forkCount>
+    <!-- Overridden at runtime! -->
+    <server.port>0000</server.port>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>flight-core</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-unix-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-kqueue</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-epoll</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-format</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-netty</artifactId>
+      <version>${project.version}</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-jdbc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>${dep.protobuf.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-api</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-vector</artifactId>
+      <version>${project.version}</version>
+      <classifier>${arrow.vector.classifier}</classifier>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.derby</groupId>
+      <artifactId>derby</artifactId>
+      <version>10.14.2.0</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-dbcp2</artifactId>
+      <version>2.7.0</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-pool2</artifactId>
+      <version>2.8.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <extensions>
+      <!-- provides os.detected.classifier (i.e. linux-x86_64, osx-x86_64) 
property -->
+      <extension>
+          <groupId>kr.motd.maven</groupId>
+          <artifactId>os-maven-plugin</artifactId>
+          <version>1.5.0.Final</version>
+      </extension>
+    </extensions>
+    <plugins>
+      <plugin>
+        <groupId>org.xolstice.maven.plugins</groupId>
+        <artifactId>protobuf-maven-plugin</artifactId>
+        <version>0.5.0</version>
+        <configuration>
+          
<protocArtifact>com.google.protobuf:protoc:${dep.protobuf.version}:exe:${os.detected.classifier}</protocArtifact>
+          <pluginId>grpc-java</pluginId>
+          
<pluginArtifact>io.grpc:protoc-gen-grpc-java:${dep.grpc.version}:exe:${os.detected.classifier}</pluginArtifact>
+        </configuration>
+        <executions>
+          <execution>
+            <id>proto-compile</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <protoSourceRoot>${basedir}/../../../format/</protoSourceRoot>
+            </configuration>
+            <goals>
+              <goal>compile</goal>
+              <goal>compile-custom</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>proto-test-compile</id>
+            <phase>generate-test-sources</phase>
+            <goals>
+              <goal>test-compile</goal>
+              <goal>test-compile-custom</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>3.2.0</version>
+        <executions>
+          <execution>
+            <id>add-sources</id>

Review comment:
       I'm not sure why this is necessary as the protobuf plugin should already 
take care of it?

##########
File path: java/flight/pom.xml
##########
@@ -0,0 +1,34 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0";
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+    <parent>
+        <artifactId>arrow-java-root</artifactId>
+        <groupId>org.apache.arrow</groupId>
+        <version>6.0.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <name>Arrow Flight</name>
+    <artifactId>arrow-flight</artifactId>
+    
<url>https://arrow.apache.org/blog/2019/10/13/introducing-arrow-flight/</url>
+
+    <packaging>pom</packaging>
+
+    <modules>

Review comment:
       It is probably not a good idea to define modules here and also at the 
top level.

##########
File path: 
java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSql.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.flight;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.emptyList;
+import static java.util.Collections.singletonList;
+import static java.util.Objects.isNull;
+import static org.apache.arrow.util.AutoCloseables.close;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.CoreMatchers.nullValue;
+
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.io.Reader;
+import java.nio.ByteBuffer;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.stream.IntStream;
+
+import org.apache.arrow.flatbuf.Message;
+import org.apache.arrow.flight.sql.FlightSqlClient;
+import org.apache.arrow.flight.sql.FlightSqlClient.PreparedStatement;
+import org.apache.arrow.flight.sql.FlightSqlProducer;
+import org.apache.arrow.flight.sql.example.FlightSqlExample;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.UInt1Vector;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.complex.DenseUnionVector;
+import org.apache.arrow.vector.ipc.message.MessageSerializer;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.arrow.vector.util.Text;
+import org.hamcrest.Matcher;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ErrorCollector;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Test direct usage of Flight SQL workflows.
+ */
+public class TestFlightSql {
+
+  protected static final Schema SCHEMA_INT_TABLE = new Schema(asList(
+      new Field("ID", new FieldType(false, MinorType.INT.getType(), null), 
null),
+      Field.nullable("KEYNAME", MinorType.VARCHAR.getType()),
+      Field.nullable("VALUE", MinorType.INT.getType()),
+      Field.nullable("FOREIGNID", MinorType.INT.getType())));
+  private static final List<List<String>> 
EXPECTED_RESULTS_FOR_STAR_SELECT_QUERY = ImmutableList.of(
+      asList("1", "one", "1", "1"), asList("2", "zero", "0", "1"), asList("3", 
"negative one", "-1", "1"));
+  private static final List<List<String>> 
EXPECTED_RESULTS_FOR_PARAMETER_BINDING = ImmutableList.of(
+      asList("1", "one", "1", "1"));
+  private static final Map<String, String> GET_SQL_INFO_EXPECTED_RESULTS_MAP = 
new LinkedHashMap<>();
+  private static final String LOCALHOST = "localhost";
+  private static final int[] ALL_SQL_INFO_ARGS = {
+      FlightSqlProducer.SqlInfo.FLIGHT_SQL_SERVER_NAME,
+      FlightSqlProducer.SqlInfo.FLIGHT_SQL_SERVER_VERSION,
+      FlightSqlProducer.SqlInfo.FLIGHT_SQL_SERVER_ARROW_VERSION,
+      FlightSqlProducer.SqlInfo.FLIGHT_SQL_SERVER_READ_ONLY,
+      FlightSqlProducer.SqlInfo.SQL_DDL_CATALOG,
+      FlightSqlProducer.SqlInfo.SQL_DDL_SCHEMA,
+      FlightSqlProducer.SqlInfo.SQL_DDL_TABLE,
+      FlightSqlProducer.SqlInfo.SQL_IDENTIFIER_CASE,
+      FlightSqlProducer.SqlInfo.SQL_IDENTIFIER_QUOTE_CHAR,
+      FlightSqlProducer.SqlInfo.SQL_QUOTED_IDENTIFIER_CASE};
+  private static int port;
+  private static BufferAllocator allocator;
+  private static FlightServer server;
+  private static FlightClient client;
+  private static FlightSqlClient sqlClient;
+  @Rule
+  public final ErrorCollector collector = new ErrorCollector();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    try (final Reader reader = new BufferedReader(
+        new FileReader("target/generated-test-resources/network.properties"))) 
{

Review comment:
       this file should actually be in the test classpath since it was added as 
a resource. Leveraging the classpath means there's no need to deal with 
relative location...

##########
File path: 
java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/StatementContext.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.flight.sql.example;
+
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.Objects;
+import java.util.Optional;
+
+import org.apache.arrow.flight.sql.FlightSqlProducer;
+import org.apache.arrow.util.AutoCloseables;
+
+/**
+ * Context for {@link T} to be persisted in memory in between {@link 
FlightSqlProducer} calls.
+ *
+ * @param <T> the {@link Statement} to be persisted.
+ */
+public final class StatementContext<T extends Statement> implements 
AutoCloseable {
+
+  private final T statement;
+  private final String query;
+
+  public StatementContext(final T statement, final String query) {
+    this.statement = Objects.requireNonNull(statement, "statement cannot be 
null.");
+    this.query = query;
+  }
+
+  public StatementContext(final T statement) {
+    this(statement, null);
+  }
+
+  /**
+   * Gets the statement wrapped by this {@link StatementContext}.
+   *
+   * @return the inner statement.
+   */
+  public T getStatement() {
+    return statement;
+  }
+
+  /**
+   * Gets the optional SQL query wrapped by this {@link StatementContext}.
+   *
+   * @return the SQL query if present; empty otherwise.
+   */
+  public Optional<String> getQuery() {
+    return Optional.ofNullable(query);
+  }
+
+  @Override
+  public void close() throws Exception {
+    Connection connection = statement.getConnection();
+    AutoCloseables.close(statement, connection);

Review comment:
       It seems potentially confusing that `StatementContext` to close both 
`statement` and `connection`. Shouldn't `connection` be closed transitively 
when `statement` is closed too instead?

##########
File path: java/pom.xml
##########
@@ -549,6 +549,12 @@
         <version>2.8.2</version>
         <scope>provided</scope>
       </dependency>
+      <dependency>
+        <groupId>org.hamcrest</groupId>
+        <artifactId>hamcrest</artifactId>
+        <version>2.2</version>
+        <scope>test</scope>

Review comment:
       Isn't the convention to leave the default scope for 
`dependencyManagement` and specify it on usage if not compile? (so that when 
used it's clear what the scope is by reading `pom.xml`)?

##########
File path: 
java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java
##########
@@ -0,0 +1,560 @@
+/*
+ * 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.flight.sql;
+
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.ActionClosePreparedStatementRequest;
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementRequest;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetCatalogs;
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.CommandGetExportedKeys;
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.CommandGetImportedKeys;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetPrimaryKeys;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSchemas;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSqlInfo;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTableTypes;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTables;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementQuery;
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementUpdate;
+import static org.apache.arrow.flight.sql.impl.FlightSql.DoPutUpdateResult;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+import org.apache.arrow.flatbuf.Message;
+import org.apache.arrow.flight.Action;
+import org.apache.arrow.flight.CallOption;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightClient;
+import org.apache.arrow.flight.FlightDescriptor;
+import org.apache.arrow.flight.FlightInfo;
+import org.apache.arrow.flight.FlightStream;
+import org.apache.arrow.flight.PutResult;
+import org.apache.arrow.flight.Result;
+import org.apache.arrow.flight.SchemaResult;
+import org.apache.arrow.flight.SyncPutListener;
+import org.apache.arrow.flight.Ticket;
+import org.apache.arrow.flight.sql.impl.FlightSql;
+import 
org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementResult;
+import 
org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery;
+import org.apache.arrow.memory.ArrowBuf;
+import org.apache.arrow.util.Preconditions;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.ipc.message.MessageSerializer;
+import org.apache.arrow.vector.types.pojo.Schema;
+
+import com.google.protobuf.Any;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.StringValue;
+
+/**
+ * Flight client with Flight SQL semantics.
+ */
+public class FlightSqlClient {
+  private final FlightClient client;
+
+  public FlightSqlClient(final FlightClient client) {
+    this.client = Objects.requireNonNull(client, "Client cannot be null!");
+  }
+
+  /**
+   * Execute a query on the server.
+   *
+   * @param query   The query to execute.
+   * @param options RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo execute(final String query, final CallOption... options) {
+    final CommandStatementQuery.Builder builder = 
CommandStatementQuery.newBuilder();
+    builder.setQuery(query);
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor, options);
+  }
+
+  /**
+   * Execute an update query on the server.
+   *
+   * @param query   The query to execute.
+   * @param options RPC-layer hints for this call.
+   * @return the number of rows affected.
+   */
+  public long executeUpdate(final String query, final CallOption... options) {
+    final CommandStatementUpdate.Builder builder = 
CommandStatementUpdate.newBuilder();
+    builder.setQuery(query);
+
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    final SyncPutListener putListener = new SyncPutListener();
+    client.startPut(descriptor, VectorSchemaRoot.of(), putListener, options);
+
+    try {
+      final PutResult read = putListener.read();
+      try (final ArrowBuf metadata = read.getApplicationMetadata()) {
+        final DoPutUpdateResult doPutUpdateResult = 
DoPutUpdateResult.parseFrom(metadata.nioBuffer());
+        return doPutUpdateResult.getRecordCount();
+      }
+    } catch (final InterruptedException | ExecutionException e) {
+      throw CallStatus.CANCELLED.withCause(e).toRuntimeException();
+    } catch (final InvalidProtocolBufferException e) {
+      throw CallStatus.INTERNAL.withCause(e).toRuntimeException();
+    }
+  }
+
+  /**
+   * Request a list of catalogs.
+   *
+   * @param options RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getCatalogs(final CallOption... options) {
+    final CommandGetCatalogs.Builder builder = CommandGetCatalogs.newBuilder();
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor, options);
+  }
+
+  /**
+   * Request a list of schemas.
+   *
+   * @param catalog             The catalog.
+   * @param schemaFilterPattern The schema filter pattern.
+   * @param options             RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getSchemas(final String catalog, final String 
schemaFilterPattern, final CallOption... options) {
+    final CommandGetSchemas.Builder builder = CommandGetSchemas.newBuilder();
+
+    if (catalog != null) {
+      builder.setCatalog(StringValue.newBuilder().setValue(catalog).build());
+    }
+
+    if (schemaFilterPattern != null) {
+      
builder.setSchemaFilterPattern(StringValue.newBuilder().setValue(schemaFilterPattern).build());
+    }
+
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor, options);
+  }
+
+  /**
+   * Get schema for a stream.
+   *
+   * @param descriptor The descriptor for the stream.
+   * @param options    RPC-layer hints for this call.
+   */
+  public SchemaResult getSchema(FlightDescriptor descriptor, CallOption... 
options) {
+    return client.getSchema(descriptor, options);
+  }
+
+  /**
+   * Retrieve a stream from the server.
+   *
+   * @param ticket  The ticket granting access to the data stream.
+   * @param options RPC-layer hints for this call.
+   */
+  public FlightStream getStream(Ticket ticket, CallOption... options) {
+    return client.getStream(ticket, options);
+  }
+
+  /**
+   * Request a set of Flight SQL metadata.
+   *
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getSqlInfo(final int... info) {

Review comment:
       As a Java developer, I would prefer a strongly typed parameter vs simply 
passing int

##########
File path: format/FlightSql.proto
##########
@@ -0,0 +1,454 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.
+ */
+
+syntax = "proto3";
+import "google/protobuf/wrappers.proto";
+import "google/protobuf/descriptor.proto";
+
+option java_package = "org.apache.arrow.flight.sql.impl";
+package arrow.flight.protocol.sql;
+
+/*
+ * Represents a metadata request. Used in the command member of 
FlightDescriptor
+ * for the following RPC calls:
+ *  - GetSchema: return the schema of the query.
+ *  - GetFlightInfo: execute the metadata request.
+ *
+ * The returned schema will be:
+ * <
+ *  info_name: uint32,
+ *  value: dense_union<string_value: string, int_value: int32, bigint_value: 
int64, int32_bitmask: int32>
+ * >
+ * where there is one row per requested piece of metadata information.
+ */
+message CommandGetSqlInfo {
+  option (experimental) = true;
+
+  /*
+   * Values are modelled after ODBC's SQLGetInfo() function. This information 
is intended to provide
+   * Flight SQL clients with basic, SQL syntax and SQL functions related 
information.
+   * More information types can be added in future releases.
+   * E.g. more SQL syntax support types, scalar functions support, type 
conversion support etc.
+   *
+   * Note that the set of metadata may expand.
+   *
+   * Initially, Flight SQL will support the following information types:
+   * - Server Information - Range [0-500)
+   * - Syntax Information - Range [500-1000)

Review comment:
       Technically there could have been since protobuf uses varint so maybe it 
could have been a case of using lower values for the most frequently accessed 
info and to create more compact messages?

##########
File path: java/flight/flight-sql/pom.xml
##########
@@ -0,0 +1,238 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>arrow-flight</artifactId>
+    <groupId>org.apache.arrow</groupId>
+    <version>6.0.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>flight-sql</artifactId>
+  <name>Arrow Flight SQL</name>
+  <description>(Experimental)Contains utility classes to expose Flight SQL 
semantics for clients and servers over Arrow Flight</description>
+  <packaging>jar</packaging>
+
+  <properties>
+    <dep.grpc.version>1.30.2</dep.grpc.version>
+    <dep.protobuf.version>3.7.1</dep.protobuf.version>
+    <forkCount>1</forkCount>
+    <!-- Overridden at runtime! -->
+    <server.port>0000</server.port>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>flight-core</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-unix-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-kqueue</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-transport-native-epoll</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-memory-netty</artifactId>
+      <version>${project.version}</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-jdbc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>${dep.protobuf.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-api</artifactId>
+      <version>${dep.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.arrow</groupId>
+      <artifactId>arrow-vector</artifactId>
+      <version>${project.version}</version>
+      <classifier>${arrow.vector.classifier}</classifier>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.derby</groupId>
+      <artifactId>derby</artifactId>
+      <version>10.14.2.0</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-dbcp2</artifactId>
+      <version>2.7.0</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-pool2</artifactId>
+      <version>2.8.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <extensions>
+      <!-- provides os.detected.classifier (i.e. linux-x86_64, osx-x86_64) 
property -->
+      <extension>
+          <groupId>kr.motd.maven</groupId>
+          <artifactId>os-maven-plugin</artifactId>
+          <version>1.5.0.Final</version>
+      </extension>
+    </extensions>
+    <plugins>
+      <plugin>
+        <groupId>org.xolstice.maven.plugins</groupId>
+        <artifactId>protobuf-maven-plugin</artifactId>
+        <version>0.5.0</version>
+        <configuration>
+          
<protocArtifact>com.google.protobuf:protoc:${dep.protobuf.version}:exe:${os.detected.classifier}</protocArtifact>
+          <pluginId>grpc-java</pluginId>
+          
<pluginArtifact>io.grpc:protoc-gen-grpc-java:${dep.grpc.version}:exe:${os.detected.classifier}</pluginArtifact>
+        </configuration>
+        <executions>

Review comment:
       Haven't the protobuf classes been already generated by flight-core?

##########
File path: 
java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSql.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.flight;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.emptyList;
+import static java.util.Collections.singletonList;
+import static java.util.Objects.isNull;
+import static org.apache.arrow.util.AutoCloseables.close;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.CoreMatchers.nullValue;
+
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.io.Reader;
+import java.nio.ByteBuffer;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.stream.IntStream;
+
+import org.apache.arrow.flatbuf.Message;
+import org.apache.arrow.flight.sql.FlightSqlClient;
+import org.apache.arrow.flight.sql.FlightSqlClient.PreparedStatement;
+import org.apache.arrow.flight.sql.FlightSqlProducer;
+import org.apache.arrow.flight.sql.example.FlightSqlExample;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.UInt1Vector;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.complex.DenseUnionVector;
+import org.apache.arrow.vector.ipc.message.MessageSerializer;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.arrow.vector.util.Text;
+import org.hamcrest.Matcher;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ErrorCollector;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Test direct usage of Flight SQL workflows.
+ */
+public class TestFlightSql {
+
+  protected static final Schema SCHEMA_INT_TABLE = new Schema(asList(
+      new Field("ID", new FieldType(false, MinorType.INT.getType(), null), 
null),
+      Field.nullable("KEYNAME", MinorType.VARCHAR.getType()),
+      Field.nullable("VALUE", MinorType.INT.getType()),
+      Field.nullable("FOREIGNID", MinorType.INT.getType())));
+  private static final List<List<String>> 
EXPECTED_RESULTS_FOR_STAR_SELECT_QUERY = ImmutableList.of(
+      asList("1", "one", "1", "1"), asList("2", "zero", "0", "1"), asList("3", 
"negative one", "-1", "1"));
+  private static final List<List<String>> 
EXPECTED_RESULTS_FOR_PARAMETER_BINDING = ImmutableList.of(
+      asList("1", "one", "1", "1"));
+  private static final Map<String, String> GET_SQL_INFO_EXPECTED_RESULTS_MAP = 
new LinkedHashMap<>();
+  private static final String LOCALHOST = "localhost";
+  private static final int[] ALL_SQL_INFO_ARGS = {
+      FlightSqlProducer.SqlInfo.FLIGHT_SQL_SERVER_NAME,
+      FlightSqlProducer.SqlInfo.FLIGHT_SQL_SERVER_VERSION,
+      FlightSqlProducer.SqlInfo.FLIGHT_SQL_SERVER_ARROW_VERSION,
+      FlightSqlProducer.SqlInfo.FLIGHT_SQL_SERVER_READ_ONLY,
+      FlightSqlProducer.SqlInfo.SQL_DDL_CATALOG,
+      FlightSqlProducer.SqlInfo.SQL_DDL_SCHEMA,
+      FlightSqlProducer.SqlInfo.SQL_DDL_TABLE,
+      FlightSqlProducer.SqlInfo.SQL_IDENTIFIER_CASE,
+      FlightSqlProducer.SqlInfo.SQL_IDENTIFIER_QUOTE_CHAR,
+      FlightSqlProducer.SqlInfo.SQL_QUOTED_IDENTIFIER_CASE};
+  private static int port;
+  private static BufferAllocator allocator;
+  private static FlightServer server;
+  private static FlightClient client;
+  private static FlightSqlClient sqlClient;
+  @Rule
+  public final ErrorCollector collector = new ErrorCollector();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    try (final Reader reader = new BufferedReader(
+        new FileReader("target/generated-test-resources/network.properties"))) 
{
+      final Properties properties = new Properties();
+      properties.load(reader);
+      port = Integer.parseInt(Objects.toString(properties.get("server.port")));
+    }
+
+    allocator = new RootAllocator(Integer.MAX_VALUE);
+
+    final Location serverLocation = Location.forGrpcInsecure(LOCALHOST, port);
+    server = FlightServer.builder(allocator, serverLocation, new 
FlightSqlExample(serverLocation))

Review comment:
       Is it actually be required to set the port in advance vs using 0 to ask 
server to take whatever ephemeral port is available, and then use 
`FlightServer#getLocation()` to get the actual location of the server? The 
second approach seems more robust as there's no guarantee that the port chosen 
during the build will still be available by the time the test is starting...

##########
File path: 
java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java
##########
@@ -0,0 +1,560 @@
+/*
+ * 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.flight.sql;
+
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.ActionClosePreparedStatementRequest;
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementRequest;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetCatalogs;
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.CommandGetExportedKeys;
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.CommandGetImportedKeys;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetPrimaryKeys;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSchemas;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSqlInfo;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTableTypes;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTables;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementQuery;
+import static 
org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementUpdate;
+import static org.apache.arrow.flight.sql.impl.FlightSql.DoPutUpdateResult;
+
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+import org.apache.arrow.flatbuf.Message;
+import org.apache.arrow.flight.Action;
+import org.apache.arrow.flight.CallOption;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightClient;
+import org.apache.arrow.flight.FlightDescriptor;
+import org.apache.arrow.flight.FlightInfo;
+import org.apache.arrow.flight.FlightStream;
+import org.apache.arrow.flight.PutResult;
+import org.apache.arrow.flight.Result;
+import org.apache.arrow.flight.SchemaResult;
+import org.apache.arrow.flight.SyncPutListener;
+import org.apache.arrow.flight.Ticket;
+import org.apache.arrow.flight.sql.impl.FlightSql;
+import 
org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementResult;
+import 
org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery;
+import org.apache.arrow.memory.ArrowBuf;
+import org.apache.arrow.util.Preconditions;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.ipc.message.MessageSerializer;
+import org.apache.arrow.vector.types.pojo.Schema;
+
+import com.google.protobuf.Any;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.StringValue;
+
+/**
+ * Flight client with Flight SQL semantics.
+ */
+public class FlightSqlClient {
+  private final FlightClient client;
+
+  public FlightSqlClient(final FlightClient client) {
+    this.client = Objects.requireNonNull(client, "Client cannot be null!");
+  }
+
+  /**
+   * Execute a query on the server.
+   *
+   * @param query   The query to execute.
+   * @param options RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo execute(final String query, final CallOption... options) {
+    final CommandStatementQuery.Builder builder = 
CommandStatementQuery.newBuilder();
+    builder.setQuery(query);
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor, options);
+  }
+
+  /**
+   * Execute an update query on the server.
+   *
+   * @param query   The query to execute.
+   * @param options RPC-layer hints for this call.
+   * @return the number of rows affected.
+   */
+  public long executeUpdate(final String query, final CallOption... options) {
+    final CommandStatementUpdate.Builder builder = 
CommandStatementUpdate.newBuilder();
+    builder.setQuery(query);
+
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    final SyncPutListener putListener = new SyncPutListener();
+    client.startPut(descriptor, VectorSchemaRoot.of(), putListener, options);
+
+    try {
+      final PutResult read = putListener.read();
+      try (final ArrowBuf metadata = read.getApplicationMetadata()) {
+        final DoPutUpdateResult doPutUpdateResult = 
DoPutUpdateResult.parseFrom(metadata.nioBuffer());
+        return doPutUpdateResult.getRecordCount();
+      }
+    } catch (final InterruptedException | ExecutionException e) {
+      throw CallStatus.CANCELLED.withCause(e).toRuntimeException();
+    } catch (final InvalidProtocolBufferException e) {
+      throw CallStatus.INTERNAL.withCause(e).toRuntimeException();
+    }
+  }
+
+  /**
+   * Request a list of catalogs.
+   *
+   * @param options RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getCatalogs(final CallOption... options) {
+    final CommandGetCatalogs.Builder builder = CommandGetCatalogs.newBuilder();
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor, options);
+  }
+
+  /**
+   * Request a list of schemas.
+   *
+   * @param catalog             The catalog.
+   * @param schemaFilterPattern The schema filter pattern.
+   * @param options             RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getSchemas(final String catalog, final String 
schemaFilterPattern, final CallOption... options) {
+    final CommandGetSchemas.Builder builder = CommandGetSchemas.newBuilder();
+
+    if (catalog != null) {
+      builder.setCatalog(StringValue.newBuilder().setValue(catalog).build());
+    }
+
+    if (schemaFilterPattern != null) {
+      
builder.setSchemaFilterPattern(StringValue.newBuilder().setValue(schemaFilterPattern).build());
+    }
+
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor, options);
+  }
+
+  /**
+   * Get schema for a stream.
+   *
+   * @param descriptor The descriptor for the stream.
+   * @param options    RPC-layer hints for this call.
+   */
+  public SchemaResult getSchema(FlightDescriptor descriptor, CallOption... 
options) {
+    return client.getSchema(descriptor, options);
+  }
+
+  /**
+   * Retrieve a stream from the server.
+   *
+   * @param ticket  The ticket granting access to the data stream.
+   * @param options RPC-layer hints for this call.
+   */
+  public FlightStream getStream(Ticket ticket, CallOption... options) {
+    return client.getStream(ticket, options);
+  }
+
+  /**
+   * Request a set of Flight SQL metadata.
+   *
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getSqlInfo(final int... info) {
+    return getSqlInfo(info, new CallOption[0]);
+  }
+
+  /**
+   * Request a set of Flight SQL metadata.
+   *
+   * @param info    The set of metadata to retrieve. None to retrieve all 
metadata.
+   * @param options RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getSqlInfo(final int[] info, final CallOption... options) {
+    return 
getSqlInfo(Arrays.stream(info).boxed().collect(Collectors.toList()), options);
+  }
+
+  /**
+   * Request a set of Flight SQL metadata.
+   *
+   * @param info    The set of metadata to retrieve. None to retrieve all 
metadata.
+   * @param options RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getSqlInfo(final List<Integer> info, final CallOption... 
options) {
+    final CommandGetSqlInfo.Builder builder = CommandGetSqlInfo.newBuilder();
+    builder.addAllInfo(info);
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor, options);
+  }
+
+  /**
+   * Request a list of tables.
+   *
+   * @param catalog             The catalog.
+   * @param schemaFilterPattern The schema filter pattern.
+   * @param tableFilterPattern  The table filter pattern.
+   * @param tableTypes          The table types to include.
+   * @param includeSchema       True to include the schema upon return, false 
to not include the schema.
+   * @param options             RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getTables(final String catalog, final String 
schemaFilterPattern,
+                              final String tableFilterPattern, final 
List<String> tableTypes,
+                              final boolean includeSchema, final CallOption... 
options) {
+    final CommandGetTables.Builder builder = CommandGetTables.newBuilder();
+
+    if (catalog != null) {
+      builder.setCatalog(StringValue.newBuilder().setValue(catalog).build());
+    }
+
+    if (schemaFilterPattern != null) {
+      
builder.setSchemaFilterPattern(StringValue.newBuilder().setValue(schemaFilterPattern).build());
+    }
+
+    if (tableFilterPattern != null) {
+      
builder.setTableNameFilterPattern(StringValue.newBuilder().setValue(tableFilterPattern).build());
+    }
+
+    if (tableTypes != null) {
+      builder.addAllTableTypes(tableTypes);
+    }
+    builder.setIncludeSchema(includeSchema);
+
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor, options);
+  }
+
+  /**
+   * Request the primary keys for a table.
+   *
+   * @param catalog The catalog.
+   * @param schema  The schema.
+   * @param table   The table.
+   * @param options RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getPrimaryKeys(final String catalog, final String schema,
+                                   final String table, final CallOption... 
options) {
+    final CommandGetPrimaryKeys.Builder builder = 
CommandGetPrimaryKeys.newBuilder();
+
+    if (catalog != null) {
+      builder.setCatalog(StringValue.newBuilder().setValue(catalog).build());
+    }
+
+    if (schema != null) {
+      builder.setSchema(StringValue.newBuilder().setValue(schema).build());
+    }
+
+    if (table != null) {
+      builder.setTable(StringValue.newBuilder().setValue(table).build());
+    }
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor, options);
+  }
+
+  /**
+   * Retrieves a description about the foreign key columns that reference the 
primary key columns of the given table.
+   *
+   * @param catalog The foreign key table catalog.
+   * @param schema  The foreign key table schema.
+   * @param table   The foreign key table. Cannot be null.
+   * @param options RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getExportedKeys(String catalog, String schema, String 
table, final CallOption... options) {
+    Objects.requireNonNull(table, "Table cannot be null.");
+
+    final CommandGetExportedKeys.Builder builder = 
CommandGetExportedKeys.newBuilder();
+
+    if (catalog != null) {
+      builder.setCatalog(StringValue.newBuilder().setValue(catalog).build());
+    }
+
+    if (schema != null) {
+      builder.setSchema(StringValue.newBuilder().setValue(schema).build());
+    }
+
+    Objects.requireNonNull(table);
+    builder.setTable(table).build();
+
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor, options);
+  }
+
+  /**
+   * Retrieves the foreign key columns for the given table.
+   *
+   * @param catalog The primary key table catalog.
+   * @param schema  The primary key table schema.
+   * @param table   The primary key table. Cannot be null.
+   * @param options RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getImportedKeys(final String catalog, final String schema, 
final String table,
+                                    final CallOption... options) {
+    Objects.requireNonNull(table, "Table cannot be null.");
+
+    final CommandGetImportedKeys.Builder builder = 
CommandGetImportedKeys.newBuilder();
+
+    if (catalog != null) {
+      builder.setCatalog(StringValue.newBuilder().setValue(catalog).build());
+    }
+
+    if (schema != null) {
+      builder.setSchema(StringValue.newBuilder().setValue(schema).build());
+    }
+
+    Objects.requireNonNull(table);
+    builder.setTable(table).build();
+
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor, options);
+  }
+
+  /**
+   * Request a list of table types.
+   *
+   * @param options RPC-layer hints for this call.
+   * @return a FlightInfo object representing the stream(s) to fetch.
+   */
+  public FlightInfo getTableTypes(final CallOption... options) {
+    final CommandGetTableTypes.Builder builder = 
CommandGetTableTypes.newBuilder();
+    final FlightDescriptor descriptor = 
FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+    return client.getInfo(descriptor, options);
+  }
+
+  /**
+   * Create a prepared statement on the server.
+   *
+   * @param query   The query to prepare.
+   * @param options RPC-layer hints for this call.
+   * @return The representation of the prepared statement which exists on the 
server.
+   */
+  public PreparedStatement prepare(final String query, final CallOption... 
options) {
+    return new PreparedStatement(client, query, options);
+  }
+
+  /**
+   * Helper class to encapsulate Flight SQL prepared statement logic.
+   */
+  public static class PreparedStatement implements AutoCloseable {
+    private final FlightClient client;
+    private final ActionCreatePreparedStatementResult preparedStatementResult;
+    private VectorSchemaRoot parameterBindingRoot;
+    private boolean isClosed;
+    private Schema resultSetSchema;
+    private Schema parameterSchema;
+
+    /**
+     * Constructor.
+     *
+     * @param client  The client. PreparedStatement does not maintain this 
resource.
+     * @param sql     The query.
+     * @param options RPC-layer hints for this call.
+     */
+    public PreparedStatement(final FlightClient client, final String sql, 
final CallOption... options) {
+      this.client = client;
+      final Action action = new Action(
+          FlightSqlUtils.FLIGHT_SQL_CREATEPREPAREDSTATEMENT.getType(),
+          Any.pack(ActionCreatePreparedStatementRequest
+                  .newBuilder()
+                  .setQuery(sql)
+                  .build())
+              .toByteArray());
+      final Iterator<Result> preparedStatementResults = 
client.doAction(action, options);
+
+      preparedStatementResult = FlightSqlUtils.unpackAndParseOrThrow(
+          preparedStatementResults.next().getBody(),
+          ActionCreatePreparedStatementResult.class);
+
+      isClosed = false;
+    }
+
+    /**
+     * Set the {@link #parameterBindingRoot} containing the parameter binding 
from a {@link PreparedStatement}
+     * operation.
+     *
+     * @param parameterBindingRoot a {@code VectorSchemaRoot} object 
containing the values to be used in the
+     *                             {@code PreparedStatement} setters.
+     */
+    public void setParameters(final VectorSchemaRoot parameterBindingRoot) {
+      if (this.parameterBindingRoot != null) {
+        if (this.parameterBindingRoot.equals(parameterBindingRoot)) {
+          return;
+        }
+        this.parameterBindingRoot.close();
+      }
+      this.parameterBindingRoot =
+          Objects.requireNonNull(parameterBindingRoot, "Parameter binding root 
cannot be null!");

Review comment:
       I guess there's an interesting question: why passing `null` should not 
invalidate the previous parameter?
   But in that scenario, it probably the check can be done first (as it is done 
usually) and the following code should be equivalent:
   ```
     Objects.requireNonNull(parameterBindingRoot, "Parameter binding root 
cannot be null!");
     if (this.parameterBindingRoot.equals(parameterBindingRoot)) {
        return;
     }
     this.parameterBindingRoot.close();
     this.parameterBindingRoot = parameterBindingRoot;
   ```

##########
File path: 
java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlExample.java
##########
@@ -0,0 +1,1733 @@
+/*
+ * 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.flight.sql.example;
+
+import static com.google.common.base.Strings.emptyToNull;
+import static com.google.protobuf.Any.pack;
+import static com.google.protobuf.ByteString.copyFrom;
+import static java.lang.String.format;
+import static java.util.Collections.singletonList;
+import static java.util.Objects.isNull;
+import static java.util.UUID.randomUUID;
+import static java.util.stream.StreamSupport.stream;
+import static 
org.apache.arrow.adapter.jdbc.JdbcToArrow.sqlToArrowVectorIterator;
+import static org.apache.arrow.adapter.jdbc.JdbcToArrowUtils.jdbcToArrowSchema;
+import static org.apache.arrow.util.Preconditions.checkState;
+import static org.slf4j.LoggerFactory.getLogger;
+
+import java.io.File;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDateTime;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.stream.Stream;
+
+import org.apache.arrow.adapter.jdbc.ArrowVectorIterator;
+import org.apache.arrow.adapter.jdbc.JdbcFieldInfo;
+import org.apache.arrow.adapter.jdbc.JdbcToArrowConfig;
+import org.apache.arrow.adapter.jdbc.JdbcToArrowUtils;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.Criteria;
+import org.apache.arrow.flight.FlightDescriptor;
+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.PutResult;
+import org.apache.arrow.flight.Result;
+import org.apache.arrow.flight.SchemaResult;
+import org.apache.arrow.flight.Ticket;
+import org.apache.arrow.flight.sql.FlightSqlProducer;
+import org.apache.arrow.flight.sql.impl.FlightSql;
+import 
org.apache.arrow.flight.sql.impl.FlightSql.ActionClosePreparedStatementRequest;
+import 
org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementRequest;
+import 
org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementResult;
+import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetCatalogs;
+import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetPrimaryKeys;
+import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSchemas;
+import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSqlInfo;
+import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTableTypes;
+import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTables;
+import 
org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery;
+import 
org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementUpdate;
+import org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementQuery;
+import org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementUpdate;
+import org.apache.arrow.memory.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.AutoCloseables;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DateMilliVector;
+import org.apache.arrow.vector.Decimal256Vector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.LargeVarCharVector;
+import org.apache.arrow.vector.SmallIntVector;
+import org.apache.arrow.vector.TimeMicroVector;
+import org.apache.arrow.vector.TimeMilliVector;
+import org.apache.arrow.vector.TimeNanoVector;
+import org.apache.arrow.vector.TimeSecVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMilliTZVector;
+import org.apache.arrow.vector.TimeStampNanoTZVector;
+import org.apache.arrow.vector.TimeStampSecTZVector;
+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.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorLoader;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.VectorUnloader;
+import org.apache.arrow.vector.complex.DenseUnionVector;
+import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.NullableVarCharHolder;
+import org.apache.arrow.vector.ipc.message.IpcOption;
+import org.apache.arrow.vector.ipc.message.MessageSerializer;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.arrow.vector.util.Text;
+import org.apache.commons.dbcp2.ConnectionFactory;
+import org.apache.commons.dbcp2.DriverManagerConnectionFactory;
+import org.apache.commons.dbcp2.PoolableConnection;
+import org.apache.commons.dbcp2.PoolableConnectionFactory;
+import org.apache.commons.dbcp2.PoolingDataSource;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.impl.GenericObjectPool;
+import org.slf4j.Logger;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Message;
+import com.google.protobuf.ProtocolStringList;
+
+/**
+ * Proof of concept {@link FlightSqlProducer} implementation showing an Apache 
Derby backed Flight SQL server capable
+ * of the following workflows:
+ * <!--
+ * TODO Revise summary: is it still matching?
+ * -->
+ * - returning a list of tables from the action `GetTables`.
+ * - creation of a prepared statement from the action 
`CreatePreparedStatement`.
+ * - execution of a prepared statement by using a {@link 
CommandPreparedStatementQuery}
+ * with {@link #getFlightInfo} and {@link #getStream}.
+ */
+public class FlightSqlExample implements FlightSqlProducer, AutoCloseable {
+  private static final IpcOption DEFAULT_OPTION = IpcOption.DEFAULT;
+  private static final String DATABASE_URI = "jdbc:derby:target/derbyDB";
+  private static final Logger LOGGER = getLogger(FlightSqlExample.class);
+  private static final Calendar DEFAULT_CALENDAR = 
JdbcToArrowUtils.getUtcCalendar();
+  private final Location location;
+  private final PoolingDataSource<PoolableConnection> dataSource;
+  private final BufferAllocator rootAllocator = new RootAllocator();
+  private final Cache<ByteString, StatementContext<PreparedStatement>> 
preparedStatementLoadingCache;
+  private final Cache<ByteString, StatementContext<Statement>> 
statementLoadingCache;
+  private final LoadingCache<ByteString, ResultSet> 
commandExecuteStatementLoadingCache;
+
+  public FlightSqlExample(final Location location) {
+    // TODO Constructor should not be doing work.
+    checkState(
+        removeDerbyDatabaseIfExists() && populateDerbyDatabase(),
+        "Failed to reset Derby database!");
+    final ConnectionFactory connectionFactory =
+        new DriverManagerConnectionFactory(DATABASE_URI, new Properties());
+    final PoolableConnectionFactory poolableConnectionFactory =
+        new PoolableConnectionFactory(connectionFactory, null);
+    final ObjectPool<PoolableConnection> connectionPool = new 
GenericObjectPool<>(poolableConnectionFactory);
+
+    poolableConnectionFactory.setPool(connectionPool);
+    // PoolingDataSource takes ownership of `connectionPool`
+    dataSource = new PoolingDataSource<>(connectionPool);
+
+    preparedStatementLoadingCache =
+        CacheBuilder.newBuilder()
+            .maximumSize(100)
+            .expireAfterWrite(10, TimeUnit.MINUTES)
+            .removalListener(new StatementRemovalListener<PreparedStatement>())
+            .build();
+
+    statementLoadingCache =
+        CacheBuilder.newBuilder()
+            .maximumSize(100)
+            .expireAfterWrite(10, TimeUnit.MINUTES)
+            .removalListener(new StatementRemovalListener<>())
+            .build();
+
+    commandExecuteStatementLoadingCache =
+        CacheBuilder.newBuilder()
+            .maximumSize(100)
+            .expireAfterWrite(10, TimeUnit.MINUTES)
+            .removalListener(new CommandExecuteStatementRemovalListener())
+            .build(new 
CommandExecuteStatementCacheLoader(statementLoadingCache));
+
+    this.location = location;
+  }
+
+  private static boolean removeDerbyDatabaseIfExists() {
+    boolean wasSuccess;
+    final Path path = Paths.get("target" + File.separator + "derbyDB");
+
+    try (final Stream<Path> walk = Files.walk(path)) {
+      /*
+       * Iterate over all paths to delete, mapping each path to the outcome of 
its own
+       * deletion as a boolean representing whether or not each individual 
operation was
+       * successful; then reduce all booleans into a single answer, and store 
that into
+       * `wasSuccess`, which will later be returned by this method.
+       * If for whatever reason the resulting `Stream<Boolean>` is empty, 
throw an `IOException`;
+       * this not expected.
+       */
+      wasSuccess = 
walk.sorted(Comparator.reverseOrder()).map(Path::toFile).map(File::delete)
+          .reduce(Boolean::logicalAnd).orElseThrow(IOException::new);
+    } catch (IOException e) {
+      /*
+       * The only acceptable scenario for an `IOException` to be thrown here 
is if
+       * an attempt to delete an non-existing file takes place -- which should 
be
+       * alright, since they would be deleted anyway.
+       */
+      if (!(wasSuccess = e instanceof NoSuchFileException)) {
+        LOGGER.error(format("Failed attempt to clear DerbyDB: <%s>", 
e.getMessage()), e);
+      }
+    }
+
+    return wasSuccess;
+  }
+
+  private static boolean populateDerbyDatabase() {
+    try (final Connection connection = 
DriverManager.getConnection("jdbc:derby:target/derbyDB;create=true");
+         Statement statement = connection.createStatement()) {
+      statement.execute("CREATE TABLE foreignTable (" +
+          "id INT not null primary key GENERATED ALWAYS AS IDENTITY (START 
WITH 1, INCREMENT BY 1), " +
+          "foreignName varchar(100), " +
+          "value int)");
+      statement.execute("CREATE TABLE intTable (" +
+          "id INT not null primary key GENERATED ALWAYS AS IDENTITY (START 
WITH 1, INCREMENT BY 1), " +
+          "keyName varchar(100), " +
+          "value int, " +
+          "foreignId int references foreignTable(id))");
+      statement.execute("INSERT INTO foreignTable (foreignName, value) VALUES 
('keyOne', 1)");
+      statement.execute("INSERT INTO foreignTable (foreignName, value) VALUES 
('keyTwo', 0)");
+      statement.execute("INSERT INTO foreignTable (foreignName, value) VALUES 
('keyThree', -1)");
+      statement.execute("INSERT INTO intTable (keyName, value, foreignId) 
VALUES ('one', 1, 1)");
+      statement.execute("INSERT INTO intTable (keyName, value, foreignId) 
VALUES ('zero', 0, 1)");
+      statement.execute("INSERT INTO intTable (keyName, value, foreignId) 
VALUES ('negative one', -1, 1)");
+    } catch (final SQLException e) {
+      LOGGER.error(format("Failed attempt to populate DerbyDB: <%s>", 
e.getMessage()), e);
+      return false;
+    }
+    return true;
+  }
+
+  private static ArrowType getArrowTypeFromJdbcType(final int jdbcDataType, 
final int precision, final int scale) {
+    final ArrowType type =
+        JdbcToArrowConfig.getDefaultJdbcToArrowTypeConverter().apply(new 
JdbcFieldInfo(jdbcDataType, precision, scale),
+            DEFAULT_CALENDAR);
+    return isNull(type) ? ArrowType.Utf8.INSTANCE : type;
+  }
+
+  private static void saveToVector(final Byte data, final UInt1Vector vector, 
final int index) {
+    vectorConsumer(
+        data,
+        vector,
+        fieldVector -> fieldVector.setNull(index),
+        (theData, fieldVector) -> fieldVector.setSafe(index, theData));
+  }
+
+  private static void saveToVector(final byte typeRegisteredId, final String 
data,
+                                   final DenseUnionVector vector, final int 
index) {
+    vectorConsumer(
+        data,
+        vector,
+        fieldVector -> {
+          // Nothing.
+        },
+        (theData, fieldVector) -> {
+          final String effectiveData = (isNull(data)) ? "" : data;
+          final NullableVarCharHolder holder = new NullableVarCharHolder();
+          final int dataLength = effectiveData.length();
+          final ArrowBuf buffer = 
fieldVector.getAllocator().buffer(dataLength);
+          buffer.writeBytes(effectiveData.getBytes(StandardCharsets.UTF_8));
+          holder.buffer = buffer;
+          holder.end = dataLength;
+          holder.isSet = 1;
+          fieldVector.setTypeId(index, typeRegisteredId);
+          fieldVector.setSafe(index, holder);
+        });
+  }
+
+  private static void saveToVector(final byte typeRegisteredId, final Integer 
data,
+                                   final DenseUnionVector vector, final int 
index) {
+    vectorConsumer(
+        data,
+        vector,
+        fieldVector -> {
+          // Nothing.
+        },
+        (theData, fieldVector) -> {
+          final NullableIntHolder holder = new NullableIntHolder();
+          holder.value = isNull(data) ? 0 : data;
+          holder.isSet = 1;
+          fieldVector.setTypeId(index, typeRegisteredId);
+          fieldVector.setSafe(index, holder);
+        });
+  }
+
+  private static void saveToVector(final Integer data, final UInt4Vector 
vector, final int index) {
+    preconditionCheckSaveToVector(vector, index);
+    vectorConsumer(data, vector, fieldVector -> fieldVector.setNull(index),
+        (theData, fieldVector) -> fieldVector.setSafe(index, data));
+  }
+
+  private static void saveToVector(final String data, final VarCharVector 
vector, final int index) {
+    preconditionCheckSaveToVector(vector, index);
+    vectorConsumer(data, vector, fieldVector -> fieldVector.setNull(index),
+        (theData, fieldVector) -> fieldVector.setSafe(index, new 
Text(theData)));
+  }
+
+  private static void saveToVector(final Integer data, final IntVector vector, 
final int index) {
+    preconditionCheckSaveToVector(vector, index);
+    vectorConsumer(data, vector, fieldVector -> fieldVector.setNull(index),
+        (theData, fieldVector) -> fieldVector.setSafe(index, theData));
+  }
+
+  private static void saveToVector(final byte[] data, final VarBinaryVector 
vector, final int index) {
+    preconditionCheckSaveToVector(vector, index);
+    vectorConsumer(data, vector, fieldVector -> fieldVector.setNull(index),
+        (theData, fieldVector) -> fieldVector.setSafe(index, theData));
+  }
+
+  private static void preconditionCheckSaveToVector(final FieldVector vector, 
final int index) {
+    Objects.requireNonNull(vector, "vector cannot be null.");
+    checkState(index >= 0, "Index must be a positive number!");
+  }
+
+  private static <T, V extends FieldVector> void vectorConsumer(final T data, 
final V vector,
+                                                                final 
Consumer<V> consumerIfNullable,
+                                                                final 
BiConsumer<T, V> defaultConsumer) {
+    if (isNull(data)) {
+      consumerIfNullable.accept(vector);
+      return;
+    }
+    defaultConsumer.accept(data, vector);
+  }
+
+  private static VectorSchemaRoot getSchemasRoot(final ResultSet data, final 
BufferAllocator allocator)
+      throws SQLException {
+    final VarCharVector catalogs = new VarCharVector("catalog_name", 
allocator);
+    final VarCharVector schemas = new VarCharVector("schema_name", allocator);
+    final List<FieldVector> vectors = ImmutableList.of(catalogs, schemas);
+    vectors.forEach(FieldVector::allocateNew);
+    final Map<FieldVector, String> vectorToColumnName = ImmutableMap.of(
+        catalogs, "TABLE_CATALOG",
+        schemas, "TABLE_SCHEM");
+    saveToVectors(vectorToColumnName, data);
+    final int rows = 
vectors.stream().map(FieldVector::getValueCount).findAny().orElseThrow(IllegalStateException::new);
+    vectors.forEach(vector -> vector.setValueCount(rows));
+    return new VectorSchemaRoot(vectors);
+  }
+
+  private static <T extends FieldVector> int saveToVectors(final Map<T, 
String> vectorToColumnName,
+                                                           final ResultSet 
data, boolean emptyToNull)
+      throws SQLException {
+    Objects.requireNonNull(vectorToColumnName, "vectorToColumnName cannot be 
null.");
+    Objects.requireNonNull(data, "data cannot be null.");
+    final Set<Entry<T, String>> entrySet = vectorToColumnName.entrySet();
+    int rows = 0;
+    for (; data.next(); rows++) {
+      for (final Entry<T, String> vectorToColumn : entrySet) {
+        final T vector = vectorToColumn.getKey();
+        final String columnName = vectorToColumn.getValue();
+        if (vector instanceof VarCharVector) {
+          String thisData = data.getString(columnName);
+          saveToVector(emptyToNull ? emptyToNull(thisData) : thisData, 
(VarCharVector) vector, rows);
+          continue;
+        } else if (vector instanceof IntVector) {
+          final int intValue = data.getInt(columnName);
+          saveToVector(data.wasNull() ? null : intValue, (IntVector) vector, 
rows);
+          continue;
+        } else if (vector instanceof UInt1Vector) {
+          final byte byteValue = data.getByte(columnName);
+          saveToVector(data.wasNull() ? null : byteValue, (UInt1Vector) 
vector, rows);
+          continue;
+        }
+        throw CallStatus.INVALID_ARGUMENT.withDescription("Provided vector not 
supported").toRuntimeException();
+      }
+    }
+    for (final Entry<T, String> vectorToColumn : entrySet) {
+      vectorToColumn.getKey().setValueCount(rows);
+    }
+
+    return rows;
+  }
+
+  private static <T extends FieldVector> void saveToVectors(final Map<T, 
String> vectorToColumnName,
+                                                            final ResultSet 
data)
+      throws SQLException {
+    saveToVectors(vectorToColumnName, data, false);
+  }
+
+  private static VectorSchemaRoot getTableTypesRoot(final ResultSet data, 
final BufferAllocator allocator)
+      throws SQLException {
+    return getRoot(data, allocator, "table_type", "TABLE_TYPE");
+  }
+
+  private static VectorSchemaRoot getCatalogsRoot(final ResultSet data, final 
BufferAllocator allocator)
+      throws SQLException {
+    return getRoot(data, allocator, "catalog_name", "TABLE_CATALOG");
+  }
+
+  private static VectorSchemaRoot getRoot(final ResultSet data, final 
BufferAllocator allocator,
+                                          final String fieldVectorName, final 
String columnName)
+      throws SQLException {
+    final VarCharVector dataVector = new VarCharVector(fieldVectorName, 
allocator);
+    saveToVectors(ImmutableMap.of(dataVector, columnName), data);
+    final int rows = dataVector.getValueCount();
+    dataVector.setValueCount(rows);
+    return new VectorSchemaRoot(singletonList(dataVector));
+  }
+
+  private static VectorSchemaRoot getTablesRoot(final DatabaseMetaData 
databaseMetaData,
+                                                final BufferAllocator 
allocator,
+                                                final boolean includeSchema,
+                                                final String catalog,
+                                                final String 
schemaFilterPattern,
+                                                final String 
tableFilterPattern,
+                                                final String... tableTypes)
+      throws SQLException, IOException {
+    /*
+     * TODO Fix DerbyDB inconsistency if possible.
+     * During the early development of this prototype, an inconsistency has 
been found in the database
+     * used for this demonstration; as DerbyDB does not operate with the 
concept of catalogs, fetching
+     * the catalog name for a given table from `DatabaseMetadata#getColumns` 
and `DatabaseMetadata#getSchemas`
+     * returns null, as expected. However, the inconsistency lies in the fact 
that accessing the same
+     * information -- that is, the catalog name for a given table -- from 
`DatabaseMetadata#getSchemas`
+     * returns an empty String.The temporary workaround for this was making 
sure we convert the empty Strings
+     * to null using `com.google.common.base.Strings#emptyToNull`.
+     */
+    Objects.requireNonNull(allocator, "BufferAllocator cannot be null.");
+    final VarCharVector catalogNameVector = new VarCharVector("catalog_name", 
allocator);
+    final VarCharVector schemaNameVector = new VarCharVector("schema_name", 
allocator);
+    final VarCharVector tableNameVector = new VarCharVector("table_name", 
allocator);
+    final VarCharVector tableTypeVector = new VarCharVector("table_type", 
allocator);
+
+    final List<FieldVector> vectors =
+        new ArrayList<>(

Review comment:
       why wrapping the immutable list into an array list? Is vectors supposed 
to be modifiable?




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to