diff --git a/format/FlightSql.proto b/format/FlightSql.proto
new file mode 100644
index 0000000000000..23ada5c6e48f6
--- /dev/null
+++ b/format/FlightSql.proto
@@ -0,0 +1,1336 @@
+/*
+ * 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
+ *
+ * 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/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 Arrow schema of the query.
+ * - GetFlightInfo: execute the metadata request.
+ *
+ * The returned Arrow schema will be:
+ * <
+ * info_name: uint32 not null,
+ * value: dense_union<
+ * string_value: utf8,
+ * bool_value: bool,
+ * bigint_value: int64,
+ * int32_bitmask: int32,
+ * string_list: list
+ * int32_to_int32_list_map: map>
+ * >
+ * 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)
+ * Range [0-10,000) is reserved for defaults (see SqlInfo enum for default options).
+ * Custom options should start at 10,000.
+ *
+ * If omitted, then all metadata will be retrieved.
+ * Flight SQL Servers may choose to include additional metadata above and beyond the specified set, however they must
+ * at least return the specified set. IDs ranging from 0 to 10,000 (exclusive) are reserved for future use.
+ * If additional metadata is included, the metadata IDs should start from 10,000.
+ */
+ repeated uint32 info = 1;
+}
+
+// Options for CommandGetSqlInfo.
+enum SqlInfo {
+
+ // Server Information [0-500): Provides basic information about the Flight SQL Server.
+
+ // Retrieves a UTF-8 string with the name of the Flight SQL Server.
+ FLIGHT_SQL_SERVER_NAME = 0;
+
+ // Retrieves a UTF-8 string with the native version of the Flight SQL Server.
+ FLIGHT_SQL_SERVER_VERSION = 1;
+
+ // Retrieves a UTF-8 string with the Arrow format version of the Flight SQL Server.
+ FLIGHT_SQL_SERVER_ARROW_VERSION = 2;
+
+ /*
+ * Retrieves a boolean value indicating whether the Flight SQL Server is read only.
+ *
+ * Returns:
+ * - false: if read-write
+ * - true: if read only
+ */
+ FLIGHT_SQL_SERVER_READ_ONLY = 3;
+
+
+ // SQL Syntax Information [500-1000): provides information about SQL syntax supported by the Flight SQL Server.
+
+ /*
+ * Retrieves a boolean value indicating whether the Flight SQL Server supports CREATE and DROP of catalogs.
+ *
+ * Returns:
+ * - false: if it doesn't support CREATE and DROP of catalogs.
+ * - true: if it supports CREATE and DROP of catalogs.
+ */
+ SQL_DDL_CATALOG = 500;
+
+ /*
+ * Retrieves a boolean value indicating whether the Flight SQL Server supports CREATE and DROP of schemas.
+ *
+ * Returns:
+ * - false: if it doesn't support CREATE and DROP of schemas.
+ * - true: if it supports CREATE and DROP of schemas.
+ */
+ SQL_DDL_SCHEMA = 501;
+
+ /*
+ * Indicates whether the Flight SQL Server supports CREATE and DROP of tables.
+ *
+ * Returns:
+ * - false: if it doesn't support CREATE and DROP of tables.
+ * - true: if it supports CREATE and DROP of tables.
+ */
+ SQL_DDL_TABLE = 502;
+
+ /*
+ * Retrieves a uint32 value representing the enu uint32 ordinal for the case sensitivity of catalog, table, schema and table names.
+ *
+ * The possible values are listed in `arrow.flight.protocol.sql.SqlSupportedCaseSensitivity`.
+ */
+ SQL_IDENTIFIER_CASE = 503;
+
+ // Retrieves a UTF-8 string with the supported character(s) used to surround a delimited identifier.
+ SQL_IDENTIFIER_QUOTE_CHAR = 504;
+
+ /*
+ * Retrieves a uint32 value representing the enu uint32 ordinal for the case sensitivity of quoted identifiers.
+ *
+ * The possible values are listed in `arrow.flight.protocol.sql.SqlSupportedCaseSensitivity`.
+ */
+ SQL_QUOTED_IDENTIFIER_CASE = 505;
+
+ /*
+ * Retrieves a boolean value indicating whether all tables are selectable.
+ *
+ * Returns:
+ * - false: if not all tables are selectable or if none are;
+ * - true: if all tables are selectable.
+ */
+ SQL_ALL_TABLES_ARE_SELECTABLE = 506;
+
+ /*
+ * Retrieves the null ordering.
+ *
+ * Returns a uint32 ordinal for the null ordering being used, as described in
+ * `arrow.flight.protocol.sql.SqlNullOrdering`.
+ */
+ SQL_NULL_ORDERING = 507;
+
+ // Retrieves a UTF-8 string list with values of the supported keywords.
+ SQL_KEYWORDS = 508;
+
+ // Retrieves a UTF-8 string list with values of the supported numeric functions.
+ SQL_NUMERIC_FUNCTIONS = 509;
+
+ // Retrieves a UTF-8 string list with values of the supported string functions.
+ SQL_STRING_FUNCTIONS = 510;
+
+ // Retrieves a UTF-8 string list with values of the supported system functions.
+ SQL_SYSTEM_FUNCTIONS = 511;
+
+ // Retrieves a UTF-8 string list with values of the supported datetime functions.
+ SQL_DATETIME_FUNCTIONS = 512;
+
+ /*
+ * Retrieves the UTF-8 string that can be used to escape wildcard characters.
+ * This is the string that can be used to escape '_' or '%' in the catalog search parameters that are a pattern
+ * (and therefore use one of the wildcard characters).
+ * The '_' character represents any single character; the '%' character represents any sequence of zero or more
+ * characters.
+ */
+ SQL_SEARCH_STRING_ESCAPE = 513;
+
+ /*
+ * Retrieves a UTF-8 string with all the "extra" characters that can be used in unquoted identifier names
+ * (those beyond a-z, A-Z, 0-9 and _).
+ */
+ SQL_EXTRA_NAME_CHARACTERS = 514;
+
+ /*
+ * Retrieves a boolean value indicating whether column aliasing is supported.
+ * If so, the SQL AS clause can be used to provide names for computed columns or to provide alias names for columns
+ * as required.
+ *
+ * Returns:
+ * - false: if column aliasing is unsupported;
+ * - true: if column aliasing is supported.
+ */
+ SQL_SUPPORTS_COLUMN_ALIASING = 515;
+
+ /*
+ * Retrieves a boolean value indicating whether concatenations between null and non-null values being
+ * null are supported.
+ *
+ * - Returns:
+ * - false: if concatenations between null and non-null values being null are unsupported;
+ * - true: if concatenations between null and non-null values being null are supported.
+ */
+ SQL_NULL_PLUS_NULL_IS_NULL = 516;
+
+ /*
+ * Retrieves a map where the key is the type to convert from and the value is a list with the types to convert to,
+ * indicating the supported conversions. Each key and each item on the list value is a value to a predefined type on
+ * SqlSupportsConvert enum.
+ * The returned map will be: map>
+ */
+ SQL_SUPPORTS_CONVERT = 517;
+
+ /*
+ * Retrieves a boolean value indicating whether, when table correlation names are supported,
+ * they are restricted to being different from the names of the tables.
+ *
+ * Returns:
+ * - false: if table correlation names are unsupported;
+ * - true: if table correlation names are supported.
+ */
+ SQL_SUPPORTS_TABLE_CORRELATION_NAMES = 518;
+
+ /*
+ * Retrieves a boolean value indicating whether, when table correlation names are supported,
+ * they are restricted to being different from the names of the tables.
+ *
+ * Returns:
+ * - false: if different table correlation names are unsupported;
+ * - true: if different table correlation names are supported
+ */
+ SQL_SUPPORTS_DIFFERENT_TABLE_CORRELATION_NAMES = 519;
+
+ /*
+ * Retrieves a boolean value indicating whether expressions in ORDER BY lists are supported.
+ *
+ * Returns:
+ * - false: if expressions in ORDER BY are unsupported;
+ * - true: if expressions in ORDER BY are supported;
+ */
+ SQL_SUPPORTS_EXPRESSIONS_IN_ORDER_BY = 520;
+
+ /*
+ * Retrieves a boolean value indicating whether using a column that is not in the SELECT statement in a GROUP BY
+ * clause is supported.
+ *
+ * Returns:
+ * - false: if using a column that is not in the SELECT statement in a GROUP BY clause is unsupported;
+ * - true: if using a column that is not in the SELECT statement in a GROUP BY clause is supported.
+ */
+ SQL_SUPPORTS_ORDER_BY_UNRELATED = 521;
+
+ /*
+ * Retrieves the supported GROUP BY commands;
+ *
+ * Returns an int32 bitmask value representing the supported commands.
+ * The returned bitmask should be parsed in order to retrieve the supported commands.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (GROUP BY is unsupported);
+ * - return 1 (\b1) => [SQL_GROUP_BY_UNRELATED];
+ * - return 2 (\b10) => [SQL_GROUP_BY_BEYOND_SELECT];
+ * - return 3 (\b11) => [SQL_GROUP_BY_UNRELATED, SQL_GROUP_BY_BEYOND_SELECT].
+ * Valid GROUP BY types are described under `arrow.flight.protocol.sql.SqlSupportedGroupBy`.
+ */
+ SQL_SUPPORTED_GROUP_BY = 522;
+
+ /*
+ * Retrieves a boolean value indicating whether specifying a LIKE escape clause is supported.
+ *
+ * Returns:
+ * - false: if specifying a LIKE escape clause is unsupported;
+ * - true: if specifying a LIKE escape clause is supported.
+ */
+ SQL_SUPPORTS_LIKE_ESCAPE_CLAUSE = 523;
+
+ /*
+ * Retrieves a boolean value indicating whether columns may be defined as non-nullable.
+ *
+ * Returns:
+ * - false: if columns cannot be defined as non-nullable;
+ * - true: if columns may be defined as non-nullable.
+ */
+ SQL_SUPPORTS_NON_NULLABLE_COLUMNS = 524;
+
+ /*
+ * Retrieves the supported SQL grammar level as per the ODBC specification.
+ *
+ * Returns an int32 bitmask value representing the supported SQL grammar level.
+ * The returned bitmask should be parsed in order to retrieve the supported grammar levels.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (SQL grammar is unsupported);
+ * - return 1 (\b1) => [SQL_MINIMUM_GRAMMAR];
+ * - return 2 (\b10) => [SQL_CORE_GRAMMAR];
+ * - return 3 (\b11) => [SQL_MINIMUM_GRAMMAR, SQL_CORE_GRAMMAR];
+ * - return 4 (\b100) => [SQL_EXTENDED_GRAMMAR];
+ * - return 5 (\b101) => [SQL_MINIMUM_GRAMMAR, SQL_EXTENDED_GRAMMAR];
+ * - return 6 (\b110) => [SQL_CORE_GRAMMAR, SQL_EXTENDED_GRAMMAR];
+ * - return 7 (\b111) => [SQL_MINIMUM_GRAMMAR, SQL_CORE_GRAMMAR, SQL_EXTENDED_GRAMMAR].
+ * Valid SQL grammar levels are described under `arrow.flight.protocol.sql.SupportedSqlGrammar`.
+ */
+ SQL_SUPPORTED_GRAMMAR = 525;
+
+ /*
+ * Retrieves the supported ANSI92 SQL grammar level.
+ *
+ * Returns an int32 bitmask value representing the supported ANSI92 SQL grammar level.
+ * The returned bitmask should be parsed in order to retrieve the supported commands.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (ANSI92 SQL grammar is unsupported);
+ * - return 1 (\b1) => [ANSI92_ENTRY_SQL];
+ * - return 2 (\b10) => [ANSI92_INTERMEDIATE_SQL];
+ * - return 3 (\b11) => [ANSI92_ENTRY_SQL, ANSI92_INTERMEDIATE_SQL];
+ * - return 4 (\b100) => [ANSI92_FULL_SQL];
+ * - return 5 (\b101) => [ANSI92_ENTRY_SQL, ANSI92_FULL_SQL];
+ * - return 6 (\b110) => [ANSI92_INTERMEDIATE_SQL, ANSI92_FULL_SQL];
+ * - return 7 (\b111) => [ANSI92_ENTRY_SQL, ANSI92_INTERMEDIATE_SQL, ANSI92_FULL_SQL].
+ * Valid ANSI92 SQL grammar levels are described under `arrow.flight.protocol.sql.SupportedAnsi92SqlGrammarLevel`.
+ */
+ SQL_ANSI92_SUPPORTED_LEVEL = 526;
+
+ /*
+ * Retrieves a boolean value indicating whether the SQL Integrity Enhancement Facility is supported.
+ *
+ * Returns:
+ * - false: if the SQL Integrity Enhancement Facility is supported;
+ * - true: if the SQL Integrity Enhancement Facility is supported.
+ */
+ SQL_SUPPORTS_INTEGRITY_ENHANCEMENT_FACILITY = 527;
+
+ /*
+ * Retrieves the support level for SQL OUTER JOINs.
+ *
+ * Returns a uint3 uint32 ordinal for the SQL ordering being used, as described in
+ * `arrow.flight.protocol.sql.SqlOuterJoinsSupportLevel`.
+ */
+ SQL_OUTER_JOINS_SUPPORT_LEVEL = 528;
+
+ // Retrieves a UTF-8 string with the preferred term for "schema".
+ SQL_SCHEMA_TERM = 529;
+
+ // Retrieves a UTF-8 string with the preferred term for "procedure".
+ SQL_PROCEDURE_TERM = 530;
+
+ // Retrieves a UTF-8 string with the preferred term for "catalog".
+ SQL_CATALOG_TERM = 531;
+
+ /*
+ * Retrieves a boolean value indicating whether a catalog appears at the start of a fully qualified table name.
+ *
+ * - false: if a catalog does not appear at the start of a fully qualified table name;
+ * - true: if a catalog appears at the start of a fully qualified table name.
+ */
+ SQL_CATALOG_AT_START = 532;
+
+ /*
+ * Retrieves the supported actions for a SQL schema.
+ *
+ * Returns an int32 bitmask value representing the supported actions for a SQL schema.
+ * The returned bitmask should be parsed in order to retrieve the supported actions for a SQL schema.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (no supported actions for SQL schema);
+ * - return 1 (\b1) => [SQL_ELEMENT_IN_PROCEDURE_CALLS];
+ * - return 2 (\b10) => [SQL_ELEMENT_IN_INDEX_DEFINITIONS];
+ * - return 3 (\b11) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_INDEX_DEFINITIONS];
+ * - return 4 (\b100) => [SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS];
+ * - return 5 (\b101) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS];
+ * - return 6 (\b110) => [SQL_ELEMENT_IN_INDEX_DEFINITIONS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS];
+ * - return 7 (\b111) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_INDEX_DEFINITIONS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS].
+ * Valid actions for a SQL schema described under `arrow.flight.protocol.sql.SqlSupportedElementActions`.
+ */
+ SQL_SCHEMAS_SUPPORTED_ACTIONS = 533;
+
+ /*
+ * Retrieves the supported actions for a SQL schema.
+ *
+ * Returns an int32 bitmask value representing the supported actions for a SQL catalog.
+ * The returned bitmask should be parsed in order to retrieve the supported actions for a SQL catalog.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (no supported actions for SQL catalog);
+ * - return 1 (\b1) => [SQL_ELEMENT_IN_PROCEDURE_CALLS];
+ * - return 2 (\b10) => [SQL_ELEMENT_IN_INDEX_DEFINITIONS];
+ * - return 3 (\b11) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_INDEX_DEFINITIONS];
+ * - return 4 (\b100) => [SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS];
+ * - return 5 (\b101) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS];
+ * - return 6 (\b110) => [SQL_ELEMENT_IN_INDEX_DEFINITIONS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS];
+ * - return 7 (\b111) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_INDEX_DEFINITIONS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS].
+ * Valid actions for a SQL catalog are described under `arrow.flight.protocol.sql.SqlSupportedElementActions`.
+ */
+ SQL_CATALOGS_SUPPORTED_ACTIONS = 534;
+
+ /*
+ * Retrieves the supported SQL positioned commands.
+ *
+ * Returns an int32 bitmask value representing the supported SQL positioned commands.
+ * The returned bitmask should be parsed in order to retrieve the supported SQL positioned commands.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (no supported SQL positioned commands);
+ * - return 1 (\b1) => [SQL_POSITIONED_DELETE];
+ * - return 2 (\b10) => [SQL_POSITIONED_UPDATE];
+ * - return 3 (\b11) => [SQL_POSITIONED_DELETE, SQL_POSITIONED_UPDATE].
+ * Valid SQL positioned commands are described under `arrow.flight.protocol.sql.SqlSupportedPositionedCommands`.
+ */
+ SQL_SUPPORTED_POSITIONED_COMMANDS = 535;
+
+ /*
+ * Retrieves a boolean value indicating whether SELECT FOR UPDATE statements are supported.
+ *
+ * Returns:
+ * - false: if SELECT FOR UPDATE statements are unsupported;
+ * - true: if SELECT FOR UPDATE statements are supported.
+ */
+ SQL_SELECT_FOR_UPDATE_SUPPORTED = 536;
+
+ /*
+ * Retrieves a boolean value indicating whether stored procedure calls that use the stored procedure escape syntax
+ * are supported.
+ *
+ * Returns:
+ * - false: if stored procedure calls that use the stored procedure escape syntax are unsupported;
+ * - true: if stored procedure calls that use the stored procedure escape syntax are supported.
+ */
+ SQL_STORED_PROCEDURES_SUPPORTED = 537;
+
+ /*
+ * Retrieves the supported SQL subqueries.
+ *
+ * Returns an int32 bitmask value representing the supported SQL subqueries.
+ * The returned bitmask should be parsed in order to retrieve the supported SQL subqueries.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (no supported SQL subqueries);
+ * - return 1 (\b1) => [SQL_SUBQUERIES_IN_COMPARISONS];
+ * - return 2 (\b10) => [SQL_SUBQUERIES_IN_EXISTS];
+ * - return 3 (\b11) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_EXISTS];
+ * - return 4 (\b100) => [SQL_SUBQUERIES_IN_INS];
+ * - return 5 (\b101) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_INS];
+ * - return 6 (\b110) => [SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_EXISTS];
+ * - return 7 (\b111) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_INS];
+ * - return 8 (\b1000) => [SQL_SUBQUERIES_IN_QUANTIFIEDS];
+ * - return 9 (\b1001) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_QUANTIFIEDS];
+ * - return 10 (\b1010) => [SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_QUANTIFIEDS];
+ * - return 11 (\b1011) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_QUANTIFIEDS];
+ * - return 12 (\b1100) => [SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_QUANTIFIEDS];
+ * - return 13 (\b1101) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_QUANTIFIEDS];
+ * - return 14 (\b1110) => [SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_QUANTIFIEDS];
+ * - return 15 (\b1111) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_QUANTIFIEDS];
+ * - ...
+ * Valid SQL subqueries are described under `arrow.flight.protocol.sql.SqlSupportedSubqueries`.
+ */
+ SQL_SUPPORTED_SUBQUERIES = 538;
+
+ /*
+ * Retrieves a boolean value indicating whether correlated subqueries are supported.
+ *
+ * Returns:
+ * - false: if correlated subqueries are unsupported;
+ * - true: if correlated subqueries are supported.
+ */
+ SQL_CORRELATED_SUBQUERIES_SUPPORTED = 539;
+
+ /*
+ * Retrieves the supported SQL UNIONs.
+ *
+ * Returns an int32 bitmask value representing the supported SQL UNIONs.
+ * The returned bitmask should be parsed in order to retrieve the supported SQL UNIONs.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (no supported SQL positioned commands);
+ * - return 1 (\b1) => [SQL_UNION];
+ * - return 2 (\b10) => [SQL_UNION_ALL];
+ * - return 3 (\b11) => [SQL_UNION, SQL_UNION_ALL].
+ * Valid SQL positioned commands are described under `arrow.flight.protocol.sql.SqlSupportedUnions`.
+ */
+ SQL_SUPPORTED_UNIONS = 540;
+
+ // Retrieves a uint32 value representing the maximum number of hex characters allowed in an inline binary literal.
+ SQL_MAX_BINARY_LITERAL_LENGTH = 541;
+
+ // Retrieves a uint32 value representing the maximum number of characters allowed for a character literal.
+ SQL_MAX_CHAR_LITERAL_LENGTH = 542;
+
+ // Retrieves a uint32 value representing the maximum number of characters allowed for a column name.
+ SQL_MAX_COLUMN_NAME_LENGTH = 543;
+
+ // Retrieves a uint32 value representing the the maximum number of columns allowed in a GROUP BY clause.
+ SQL_MAX_COLUMNS_IN_GROUP_BY = 544;
+
+ // Retrieves a uint32 value representing the maximum number of columns allowed in an index.
+ SQL_MAX_COLUMNS_IN_INDEX = 545;
+
+ // Retrieves a uint32 value representing the maximum number of columns allowed in an ORDER BY clause.
+ SQL_MAX_COLUMNS_IN_ORDER_BY = 546;
+
+ // Retrieves a uint32 value representing the maximum number of columns allowed in a SELECT list.
+ SQL_MAX_COLUMNS_IN_SELECT = 547;
+
+ // Retrieves a uint32 value representing the maximum number of columns allowed in a table.
+ SQL_MAX_COLUMNS_IN_TABLE = 548;
+
+ // Retrieves a uint32 value representing the maximum number of concurrent connections possible.
+ SQL_MAX_CONNECTIONS = 549;
+
+ // Retrieves a uint32 value the maximum number of characters allowed in a cursor name.
+ SQL_MAX_CURSOR_NAME_LENGTH = 550;
+
+ /*
+ * Retrieves a uint32 value representing the maximum number of bytes allowed for an index,
+ * including all of the parts of the index.
+ */
+ SQL_MAX_INDEX_LENGTH = 551;
+
+ // Retrieves a uint32 value representing the maximum number of characters allowed in a schema name.
+ SQL_DB_SCHEMA_NAME_LENGTH = 552;
+
+ // Retrieves a uint32 value representing the maximum number of characters allowed in a procedure name.
+ SQL_MAX_PROCEDURE_NAME_LENGTH = 553;
+
+ // Retrieves a uint32 value representing the maximum number of characters allowed in a catalog name.
+ SQL_MAX_CATALOG_NAME_LENGTH = 554;
+
+ // Retrieves a uint32 value representing the maximum number of bytes allowed in a single row.
+ SQL_MAX_ROW_SIZE = 555;
+
+ /*
+ * Retrieves a boolean indicating whether the return value for the JDBC method getMaxRowSize includes the SQL
+ * data types LONGVARCHAR and LONGVARBINARY.
+ *
+ * Returns:
+ * - false: if return value for the JDBC method getMaxRowSize does
+ * not include the SQL data types LONGVARCHAR and LONGVARBINARY;
+ * - true: if return value for the JDBC method getMaxRowSize includes
+ * the SQL data types LONGVARCHAR and LONGVARBINARY.
+ */
+ SQL_MAX_ROW_SIZE_INCLUDES_BLOBS = 556;
+
+ /*
+ * Retrieves a uint32 value representing the maximum number of characters allowed for an SQL statement;
+ * a result of 0 (zero) means that there is no limit or the limit is not known.
+ */
+ SQL_MAX_STATEMENT_LENGTH = 557;
+
+ // Retrieves a uint32 value representing the maximum number of active statements that can be open at the same time.
+ SQL_MAX_STATEMENTS = 558;
+
+ // Retrieves a uint32 value representing the maximum number of characters allowed in a table name.
+ SQL_MAX_TABLE_NAME_LENGTH = 559;
+
+ // Retrieves a uint32 value representing the maximum number of tables allowed in a SELECT statement.
+ SQL_MAX_TABLES_IN_SELECT = 560;
+
+ // Retrieves a uint32 value representing the maximum number of characters allowed in a user name.
+ SQL_MAX_USERNAME_LENGTH = 561;
+
+ /*
+ * Retrieves this database's default transaction isolation level as described in
+ * `arrow.flight.protocol.sql.SqlTransactionIsolationLevel`.
+ *
+ * Returns a uint32 ordinal for the SQL transaction isolation level.
+ */
+ SQL_DEFAULT_TRANSACTION_ISOLATION = 562;
+
+ /*
+ * Retrieves a boolean value indicating whether transactions are supported. If not, invoking the method commit is a
+ * noop, and the isolation level is `arrow.flight.protocol.sql.SqlTransactionIsolationLevel.TRANSACTION_NONE`.
+ *
+ * Returns:
+ * - false: if transactions are unsupported;
+ * - true: if transactions are supported.
+ */
+ SQL_TRANSACTIONS_SUPPORTED = 563;
+
+ /*
+ * Retrieves the supported transactions isolation levels.
+ *
+ * Returns an int32 bitmask value representing the supported transactions isolation levels.
+ * The returned bitmask should be parsed in order to retrieve the supported transactions isolation levels.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (no supported SQL transactions isolation levels);
+ * - return 1 (\b1) => [SQL_TRANSACTION_NONE];
+ * - return 2 (\b10) => [SQL_TRANSACTION_READ_UNCOMMITTED];
+ * - return 3 (\b11) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_READ_UNCOMMITTED];
+ * - return 4 (\b100) => [SQL_TRANSACTION_REPEATABLE_READ];
+ * - return 5 (\b101) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_REPEATABLE_READ];
+ * - return 6 (\b110) => [SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ];
+ * - return 7 (\b111) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ];
+ * - return 8 (\b1000) => [SQL_TRANSACTION_REPEATABLE_READ];
+ * - return 9 (\b1001) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_REPEATABLE_READ];
+ * - return 10 (\b1010) => [SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ];
+ * - return 11 (\b1011) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ];
+ * - return 12 (\b1100) => [SQL_TRANSACTION_REPEATABLE_READ, SQL_TRANSACTION_REPEATABLE_READ];
+ * - return 13 (\b1101) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_REPEATABLE_READ, SQL_TRANSACTION_REPEATABLE_READ];
+ * - return 14 (\b1110) => [SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ, SQL_TRANSACTION_REPEATABLE_READ];
+ * - return 15 (\b1111) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ, SQL_TRANSACTION_REPEATABLE_READ];
+ * - return 16 (\b10000) => [SQL_TRANSACTION_SERIALIZABLE];
+ * - ...
+ * Valid SQL positioned commands are described under `arrow.flight.protocol.sql.SqlTransactionIsolationLevel`.
+ */
+ SQL_SUPPORTED_TRANSACTIONS_ISOLATION_LEVELS = 564;
+
+ /*
+ * Retrieves a boolean value indicating whether a data definition statement within a transaction forces
+ * the transaction to commit.
+ *
+ * Returns:
+ * - false: if a data definition statement within a transaction does not force the transaction to commit;
+ * - true: if a data definition statement within a transaction forces the transaction to commit.
+ */
+ SQL_DATA_DEFINITION_CAUSES_TRANSACTION_COMMIT = 565;
+
+ /*
+ * Retrieves a boolean value indicating whether a data definition statement within a transaction is ignored.
+ *
+ * Returns:
+ * - false: if a data definition statement within a transaction is taken into account;
+ * - true: a data definition statement within a transaction is ignored.
+ */
+ SQL_DATA_DEFINITIONS_IN_TRANSACTIONS_IGNORED = 566;
+
+ /*
+ * Retrieves an int32 bitmask value representing the supported result set types.
+ * The returned bitmask should be parsed in order to retrieve the supported result set types.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (no supported result set types);
+ * - return 1 (\b1) => [SQL_RESULT_SET_TYPE_UNSPECIFIED];
+ * - return 2 (\b10) => [SQL_RESULT_SET_TYPE_FORWARD_ONLY];
+ * - return 3 (\b11) => [SQL_RESULT_SET_TYPE_UNSPECIFIED, SQL_RESULT_SET_TYPE_FORWARD_ONLY];
+ * - return 4 (\b100) => [SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE];
+ * - return 5 (\b101) => [SQL_RESULT_SET_TYPE_UNSPECIFIED, SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE];
+ * - return 6 (\b110) => [SQL_RESULT_SET_TYPE_FORWARD_ONLY, SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE];
+ * - return 7 (\b111) => [SQL_RESULT_SET_TYPE_UNSPECIFIED, SQL_RESULT_SET_TYPE_FORWARD_ONLY, SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE];
+ * - return 8 (\b1000) => [SQL_RESULT_SET_TYPE_SCROLL_SENSITIVE];
+ * - ...
+ * Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetType`.
+ */
+ SQL_SUPPORTED_RESULT_SET_TYPES = 567;
+
+ /*
+ * Returns an int32 bitmask value concurrency types supported for
+ * `arrow.flight.protocol.sql.SqlSupportedResultSetType.SQL_RESULT_SET_TYPE_UNSPECIFIED`.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (no supported concurrency types for this result set type)
+ * - return 1 (\b1) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED]
+ * - return 2 (\b10) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY]
+ * - return 3 (\b11) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY]
+ * - return 4 (\b100) => [SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * - return 5 (\b101) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * - return 6 (\b110) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * - return 7 (\b111) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency`.
+ */
+ SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_UNSPECIFIED = 568;
+
+ /*
+ * Returns an int32 bitmask value concurrency types supported for
+ * `arrow.flight.protocol.sql.SqlSupportedResultSetType.SQL_RESULT_SET_TYPE_FORWARD_ONLY`.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (no supported concurrency types for this result set type)
+ * - return 1 (\b1) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED]
+ * - return 2 (\b10) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY]
+ * - return 3 (\b11) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY]
+ * - return 4 (\b100) => [SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * - return 5 (\b101) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * - return 6 (\b110) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * - return 7 (\b111) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency`.
+ */
+ SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_FORWARD_ONLY = 569;
+
+ /*
+ * Returns an int32 bitmask value concurrency types supported for
+ * `arrow.flight.protocol.sql.SqlSupportedResultSetType.SQL_RESULT_SET_TYPE_SCROLL_SENSITIVE`.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (no supported concurrency types for this result set type)
+ * - return 1 (\b1) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED]
+ * - return 2 (\b10) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY]
+ * - return 3 (\b11) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY]
+ * - return 4 (\b100) => [SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * - return 5 (\b101) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * - return 6 (\b110) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * - return 7 (\b111) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency`.
+ */
+ SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_SCROLL_SENSITIVE = 570;
+
+ /*
+ * Returns an int32 bitmask value concurrency types supported for
+ * `arrow.flight.protocol.sql.SqlSupportedResultSetType.SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE`.
+ *
+ * For instance:
+ * - return 0 (\b0) => [] (no supported concurrency types for this result set type)
+ * - return 1 (\b1) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED]
+ * - return 2 (\b10) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY]
+ * - return 3 (\b11) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY]
+ * - return 4 (\b100) => [SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * - return 5 (\b101) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * - return 6 (\b110) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * - return 7 (\b111) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE]
+ * Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency`.
+ */
+ SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_SCROLL_INSENSITIVE = 571;
+
+ /*
+ * Retrieves a boolean value indicating whether this database supports batch updates.
+ *
+ * - false: if this database does not support batch updates;
+ * - true: if this database supports batch updates.
+ */
+ SQL_BATCH_UPDATES_SUPPORTED = 572;
+
+ /*
+ * Retrieves a boolean value indicating whether this database supports savepoints.
+ *
+ * Returns:
+ * - false: if this database does not support savepoints;
+ * - true: if this database supports savepoints.
+ */
+ SQL_SAVEPOINTS_SUPPORTED = 573;
+
+ /*
+ * Retrieves a boolean value indicating whether named parameters are supported in callable statements.
+ *
+ * Returns:
+ * - false: if named parameters in callable statements are unsupported;
+ * - true: if named parameters in callable statements are supported.
+ */
+ SQL_NAMED_PARAMETERS_SUPPORTED = 574;
+
+ /*
+ * Retrieves a boolean value indicating whether updates made to a LOB are made on a copy or directly to the LOB.
+ *
+ * Returns:
+ * - false: if updates made to a LOB are made directly to the LOB;
+ * - true: if updates made to a LOB are made on a copy.
+ */
+ SQL_LOCATORS_UPDATE_COPY = 575;
+
+ /*
+ * Retrieves a boolean value indicating whether invoking user-defined or vendor functions
+ * using the stored procedure escape syntax is supported.
+ *
+ * Returns:
+ * - false: if invoking user-defined or vendor functions using the stored procedure escape syntax is unsupported;
+ * - true: if invoking user-defined or vendor functions using the stored procedure escape syntax is supported.
+ */
+ SQL_STORED_FUNCTIONS_USING_CALL_SYNTAX_SUPPORTED = 576;
+}
+
+enum SqlSupportedCaseSensitivity {
+ SQL_CASE_SENSITIVITY_UNKNOWN = 0;
+ SQL_CASE_SENSITIVITY_CASE_INSENSITIVE = 1;
+ SQL_CASE_SENSITIVITY_UPPERCASE = 2;
+ SQL_CASE_SENSITIVITY_LOWERCASE = 3;
+}
+
+enum SqlNullOrdering {
+ SQL_NULLS_SORTED_HIGH = 0;
+ SQL_NULLS_SORTED_LOW = 1;
+ SQL_NULLS_SORTED_AT_START = 2;
+ SQL_NULLS_SORTED_AT_END = 3;
+}
+
+enum SupportedSqlGrammar {
+ SQL_MINIMUM_GRAMMAR = 0;
+ SQL_CORE_GRAMMAR = 1;
+ SQL_EXTENDED_GRAMMAR = 2;
+}
+
+enum SupportedAnsi92SqlGrammarLevel {
+ ANSI92_ENTRY_SQL = 0;
+ ANSI92_INTERMEDIATE_SQL = 1;
+ ANSI92_FULL_SQL = 2;
+}
+
+enum SqlOuterJoinsSupportLevel {
+ SQL_JOINS_UNSUPPORTED = 0;
+ SQL_LIMITED_OUTER_JOINS = 1;
+ SQL_FULL_OUTER_JOINS = 2;
+}
+
+enum SqlSupportedGroupBy {
+ SQL_GROUP_BY_UNRELATED = 0;
+ SQL_GROUP_BY_BEYOND_SELECT = 1;
+}
+
+enum SqlSupportedElementActions {
+ SQL_ELEMENT_IN_PROCEDURE_CALLS = 0;
+ SQL_ELEMENT_IN_INDEX_DEFINITIONS = 1;
+ SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS = 2;
+}
+
+enum SqlSupportedPositionedCommands {
+ SQL_POSITIONED_DELETE = 0;
+ SQL_POSITIONED_UPDATE = 1;
+}
+
+enum SqlSupportedSubqueries {
+ SQL_SUBQUERIES_IN_COMPARISONS = 0;
+ SQL_SUBQUERIES_IN_EXISTS = 1;
+ SQL_SUBQUERIES_IN_INS = 2;
+ SQL_SUBQUERIES_IN_QUANTIFIEDS = 3;
+}
+
+enum SqlSupportedUnions {
+ SQL_UNION = 0;
+ SQL_UNION_ALL = 1;
+}
+
+enum SqlTransactionIsolationLevel {
+ SQL_TRANSACTION_NONE = 0;
+ SQL_TRANSACTION_READ_UNCOMMITTED = 1;
+ SQL_TRANSACTION_READ_COMMITTED = 2;
+ SQL_TRANSACTION_REPEATABLE_READ = 3;
+ SQL_TRANSACTION_SERIALIZABLE = 4;
+}
+
+enum SqlSupportedTransactions {
+ SQL_TRANSACTION_UNSPECIFIED = 0;
+ SQL_DATA_DEFINITION_TRANSACTIONS = 1;
+ SQL_DATA_MANIPULATION_TRANSACTIONS = 2;
+}
+
+enum SqlSupportedResultSetType {
+ SQL_RESULT_SET_TYPE_UNSPECIFIED = 0;
+ SQL_RESULT_SET_TYPE_FORWARD_ONLY = 1;
+ SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE = 2;
+ SQL_RESULT_SET_TYPE_SCROLL_SENSITIVE = 3;
+}
+
+enum SqlSupportedResultSetConcurrency {
+ SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED = 0;
+ SQL_RESULT_SET_CONCURRENCY_READ_ONLY = 1;
+ SQL_RESULT_SET_CONCURRENCY_UPDATABLE = 2;
+}
+
+enum SqlSupportsConvert {
+ SQL_CONVERT_BIGINT = 0;
+ SQL_CONVERT_BINARY = 1;
+ SQL_CONVERT_BIT = 2;
+ SQL_CONVERT_CHAR = 3;
+ SQL_CONVERT_DATE = 4;
+ SQL_CONVERT_DECIMAL = 5;
+ SQL_CONVERT_FLOAT = 6;
+ SQL_CONVERT_INTEGER = 7;
+ SQL_CONVERT_INTERVAL_DAY_TIME = 8;
+ SQL_CONVERT_INTERVAL_YEAR_MONTH = 9;
+ SQL_CONVERT_LONGVARBINARY = 10;
+ SQL_CONVERT_LONGVARCHAR = 11;
+ SQL_CONVERT_NUMERIC = 12;
+ SQL_CONVERT_REAL = 13;
+ SQL_CONVERT_SMALLINT = 14;
+ SQL_CONVERT_TIME = 15;
+ SQL_CONVERT_TIMESTAMP = 16;
+ SQL_CONVERT_TINYINT = 17;
+ SQL_CONVERT_VARBINARY = 18;
+ SQL_CONVERT_VARCHAR = 19;
+}
+
+/*
+ * Represents a request to retrieve the list of catalogs on a Flight SQL enabled backend.
+ * The definition of a catalog depends on vendor/implementation. It is usually the database itself
+ * Used in the command member of FlightDescriptor for the following RPC calls:
+ * - GetSchema: return the Arrow schema of the query.
+ * - GetFlightInfo: execute the catalog metadata request.
+ *
+ * The returned Arrow schema will be:
+ * <
+ * catalog_name: utf8 not null
+ * >
+ * The returned data should be ordered by catalog_name.
+ */
+message CommandGetCatalogs {
+ option (experimental) = true;
+}
+
+/*
+ * Represents a request to retrieve the list of database schemas on a Flight SQL enabled backend.
+ * The definition of a database schema depends on vendor/implementation. It is usually a collection of tables.
+ * Used in the command member of FlightDescriptor for the following RPC calls:
+ * - GetSchema: return the Arrow schema of the query.
+ * - GetFlightInfo: execute the catalog metadata request.
+ *
+ * The returned Arrow schema will be:
+ * <
+ * catalog_name: utf8,
+ * db_schema_name: utf8 not null
+ * >
+ * The returned data should be ordered by catalog_name, then db_schema_name.
+ */
+message CommandGetDbSchemas {
+ option (experimental) = true;
+
+ /*
+ * Specifies the Catalog to search for the tables.
+ * An empty string retrieves those without a catalog.
+ * If omitted the catalog name should not be used to narrow the search.
+ */
+ optional string catalog = 1;
+
+ /*
+ * Specifies a filter pattern for schemas to search for.
+ * When no db_schema_filter_pattern is provided, the pattern will not be used to narrow the search.
+ * In the pattern string, two special characters can be used to denote matching rules:
+ * - "%" means to match any substring with 0 or more characters.
+ * - "_" means to match any one character.
+ */
+ optional string db_schema_filter_pattern = 2;
+}
+
+/*
+ * Represents a request to retrieve the list of tables, and optionally their schemas, on a Flight SQL enabled backend.
+ * Used in the command member of FlightDescriptor for the following RPC calls:
+ * - GetSchema: return the Arrow schema of the query.
+ * - GetFlightInfo: execute the catalog metadata request.
+ *
+ * The returned Arrow schema will be:
+ * <
+ * catalog_name: utf8,
+ * db_schema_name: utf8,
+ * table_name: utf8 not null,
+ * table_type: utf8 not null,
+ * [optional] table_schema: bytes not null (schema of the table as described in Schema.fbs::Schema,
+ * it is serialized as an IPC message.)
+ * >
+ * The returned data should be ordered by catalog_name, db_schema_name, table_name, then table_type, followed by table_schema if requested.
+ */
+message CommandGetTables {
+ option (experimental) = true;
+
+ /*
+ * Specifies the Catalog to search for the tables.
+ * An empty string retrieves those without a catalog.
+ * If omitted the catalog name should not be used to narrow the search.
+ */
+ optional string catalog = 1;
+
+ /*
+ * Specifies a filter pattern for schemas to search for.
+ * When no db_schema_filter_pattern is provided, all schemas matching other filters are searched.
+ * In the pattern string, two special characters can be used to denote matching rules:
+ * - "%" means to match any substring with 0 or more characters.
+ * - "_" means to match any one character.
+ */
+ optional string db_schema_filter_pattern = 2;
+
+ /*
+ * Specifies a filter pattern for tables to search for.
+ * When no table_name_filter_pattern is provided, all tables matching other filters are searched.
+ * In the pattern string, two special characters can be used to denote matching rules:
+ * - "%" means to match any substring with 0 or more characters.
+ * - "_" means to match any one character.
+ */
+ optional string table_name_filter_pattern = 3;
+
+ /*
+ * Specifies a filter of table types which must match.
+ * The table types depend on vendor/implementation. It is usually used to separate tables from views or system tables.
+ * TABLE, VIEW, and SYSTEM TABLE are commonly supported.
+ */
+ repeated string table_types = 4;
+
+ // Specifies if the Arrow schema should be returned for found tables.
+ bool include_schema = 5;
+}
+
+/*
+ * Represents a request to retrieve the list of table types on a Flight SQL enabled backend.
+ * The table types depend on vendor/implementation. It is usually used to separate tables from views or system tables.
+ * TABLE, VIEW, and SYSTEM TABLE are commonly supported.
+ * Used in the command member of FlightDescriptor for the following RPC calls:
+ * - GetSchema: return the Arrow schema of the query.
+ * - GetFlightInfo: execute the catalog metadata request.
+ *
+ * The returned Arrow schema will be:
+ * <
+ * table_type: utf8 not null
+ * >
+ * The returned data should be ordered by table_type.
+ */
+message CommandGetTableTypes {
+ option (experimental) = true;
+}
+
+/*
+ * Represents a request to retrieve the primary keys of a table on a Flight SQL enabled backend.
+ * Used in the command member of FlightDescriptor for the following RPC calls:
+ * - GetSchema: return the Arrow schema of the query.
+ * - GetFlightInfo: execute the catalog metadata request.
+ *
+ * The returned Arrow schema will be:
+ * <
+ * catalog_name: utf8,
+ * db_schema_name: utf8,
+ * table_name: utf8 not null,
+ * column_name: utf8 not null,
+ * key_name: utf8,
+ * key_sequence: int not null
+ * >
+ * The returned data should be ordered by catalog_name, db_schema_name, table_name, key_name, then key_sequence.
+ */
+message CommandGetPrimaryKeys {
+ option (experimental) = true;
+
+ /*
+ * Specifies the catalog to search for the table.
+ * An empty string retrieves those without a catalog.
+ * If omitted the catalog name should not be used to narrow the search.
+ */
+ optional string catalog = 1;
+
+ /*
+ * Specifies the schema to search for the table.
+ * An empty string retrieves those without a schema.
+ * If omitted the schema name should not be used to narrow the search.
+ */
+ optional string db_schema = 2;
+
+ // Specifies the table to get the primary keys for.
+ string table = 3;
+}
+
+enum UpdateDeleteRules {
+ CASCADE = 0;
+ RESTRICT = 1;
+ SET_NULL = 2;
+ NO_ACTION = 3;
+ SET_DEFAULT = 4;
+}
+
+/*
+ * Represents a request to retrieve a description of the foreign key columns that reference the given table's
+ * primary key columns (the foreign keys exported by a table) of a table on a Flight SQL enabled backend.
+ * Used in the command member of FlightDescriptor for the following RPC calls:
+ * - GetSchema: return the Arrow schema of the query.
+ * - GetFlightInfo: execute the catalog metadata request.
+ *
+ * The returned Arrow schema will be:
+ * <
+ * pk_catalog_name: utf8,
+ * pk_db_schema_name: utf8,
+ * pk_table_name: utf8 not null,
+ * pk_column_name: utf8 not null,
+ * fk_catalog_name: utf8,
+ * fk_db_schema_name: utf8,
+ * fk_table_name: utf8 not null,
+ * fk_column_name: utf8 not null,
+ * key_sequence: int not null,
+ * fk_key_name: utf8,
+ * pk_key_name: utf8,
+ * update_rule: uint1 not null,
+ * delete_rule: uint1 not null
+ * >
+ * The returned data should be ordered by fk_catalog_name, fk_db_schema_name, fk_table_name, fk_key_name, then key_sequence.
+ * update_rule and delete_rule returns a byte that is equivalent to actions declared on UpdateDeleteRules enum.
+ */
+message CommandGetExportedKeys {
+ option (experimental) = true;
+
+ /*
+ * Specifies the catalog to search for the foreign key table.
+ * An empty string retrieves those without a catalog.
+ * If omitted the catalog name should not be used to narrow the search.
+ */
+ optional string catalog = 1;
+
+ /*
+ * Specifies the schema to search for the foreign key table.
+ * An empty string retrieves those without a schema.
+ * If omitted the schema name should not be used to narrow the search.
+ */
+ optional string db_schema = 2;
+
+ // Specifies the foreign key table to get the foreign keys for.
+ string table = 3;
+}
+
+/*
+ * Represents a request to retrieve the foreign keys of a table on a Flight SQL enabled backend.
+ * Used in the command member of FlightDescriptor for the following RPC calls:
+ * - GetSchema: return the Arrow schema of the query.
+ * - GetFlightInfo: execute the catalog metadata request.
+ *
+ * The returned Arrow schema will be:
+ * <
+ * pk_catalog_name: utf8,
+ * pk_db_schema_name: utf8,
+ * pk_table_name: utf8 not null,
+ * pk_column_name: utf8 not null,
+ * fk_catalog_name: utf8,
+ * fk_db_schema_name: utf8,
+ * fk_table_name: utf8 not null,
+ * fk_column_name: utf8 not null,
+ * key_sequence: int not null,
+ * fk_key_name: utf8,
+ * pk_key_name: utf8,
+ * update_rule: uint1 not null,
+ * delete_rule: uint1 not null
+ * >
+ * The returned data should be ordered by pk_catalog_name, pk_db_schema_name, pk_table_name, pk_key_name, then key_sequence.
+ * update_rule and delete_rule returns a byte that is equivalent to actions:
+ * - 0 = CASCADE
+ * - 1 = RESTRICT
+ * - 2 = SET NULL
+ * - 3 = NO ACTION
+ * - 4 = SET DEFAULT
+ */
+message CommandGetImportedKeys {
+ option (experimental) = true;
+
+ /*
+ * Specifies the catalog to search for the primary key table.
+ * An empty string retrieves those without a catalog.
+ * If omitted the catalog name should not be used to narrow the search.
+ */
+ optional string catalog = 1;
+
+ /*
+ * Specifies the schema to search for the primary key table.
+ * An empty string retrieves those without a schema.
+ * If omitted the schema name should not be used to narrow the search.
+ */
+ optional string db_schema = 2;
+
+ // Specifies the primary key table to get the foreign keys for.
+ string table = 3;
+}
+
+/*
+ * Represents a request to retrieve a description of the foreign key columns in the given foreign key table that
+ * reference the primary key or the columns representing a unique constraint of the parent table (could be the same
+ * or a different table) on a Flight SQL enabled backend.
+ * Used in the command member of FlightDescriptor for the following RPC calls:
+ * - GetSchema: return the Arrow schema of the query.
+ * - GetFlightInfo: execute the catalog metadata request.
+ *
+ * The returned Arrow schema will be:
+ * <
+ * pk_catalog_name: utf8,
+ * pk_db_schema_name: utf8,
+ * pk_table_name: utf8 not null,
+ * pk_column_name: utf8 not null,
+ * fk_catalog_name: utf8,
+ * fk_db_schema_name: utf8,
+ * fk_table_name: utf8 not null,
+ * fk_column_name: utf8 not null,
+ * key_sequence: int not null,
+ * fk_key_name: utf8,
+ * pk_key_name: utf8,
+ * update_rule: uint1 not null,
+ * delete_rule: uint1 not null
+ * >
+ * The returned data should be ordered by pk_catalog_name, pk_db_schema_name, pk_table_name, pk_key_name, then key_sequence.
+ * update_rule and delete_rule returns a byte that is equivalent to actions:
+ * - 0 = CASCADE
+ * - 1 = RESTRICT
+ * - 2 = SET NULL
+ * - 3 = NO ACTION
+ * - 4 = SET DEFAULT
+ */
+message CommandGetCrossReference {
+ option (experimental) = true;
+
+ /**
+ * The catalog name where the parent table is.
+ * An empty string retrieves those without a catalog.
+ * If omitted the catalog name should not be used to narrow the search.
+ */
+ optional string pk_catalog = 1;
+
+ /**
+ * The Schema name where the parent table is.
+ * An empty string retrieves those without a schema.
+ * If omitted the schema name should not be used to narrow the search.
+ */
+ optional string pk_db_schema = 2;
+
+ /**
+ * The parent table name. It cannot be null.
+ */
+ string pk_table = 3;
+
+ /**
+ * The catalog name where the foreign table is.
+ * An empty string retrieves those without a catalog.
+ * If omitted the catalog name should not be used to narrow the search.
+ */
+ optional string fk_catalog = 4;
+
+ /**
+ * The schema name where the foreign table is.
+ * An empty string retrieves those without a schema.
+ * If omitted the schema name should not be used to narrow the search.
+ */
+ optional string fk_db_schema = 5;
+
+ /**
+ * The foreign table name. It cannot be null.
+ */
+ string fk_table = 6;
+}
+
+// SQL Execution Action Messages
+
+/*
+ * Request message for the "CreatePreparedStatement" action on a Flight SQL enabled backend.
+ */
+message ActionCreatePreparedStatementRequest {
+ option (experimental) = true;
+
+ // The valid SQL string to create a prepared statement for.
+ string query = 1;
+}
+
+/*
+ * Wrap the result of a "GetPreparedStatement" action.
+ *
+ * The resultant PreparedStatement can be closed either:
+ * - Manually, through the "ClosePreparedStatement" action;
+ * - Automatically, by a server timeout.
+ */
+message ActionCreatePreparedStatementResult {
+ option (experimental) = true;
+
+ // Opaque handle for the prepared statement on the server.
+ bytes prepared_statement_handle = 1;
+
+ // If a result set generating query was provided, dataset_schema contains the
+ // schema of the dataset as described in Schema.fbs::Schema, it is serialized as an IPC message.
+ bytes dataset_schema = 2;
+
+ // If the query provided contained parameters, parameter_schema contains the
+ // schema of the expected parameters as described in Schema.fbs::Schema, it is serialized as an IPC message.
+ bytes parameter_schema = 3;
+}
+
+/*
+ * Request message for the "ClosePreparedStatement" action on a Flight SQL enabled backend.
+ * Closes server resources associated with the prepared statement handle.
+ */
+message ActionClosePreparedStatementRequest {
+ option (experimental) = true;
+
+ // Opaque handle for the prepared statement on the server.
+ bytes prepared_statement_handle = 1;
+}
+
+
+// SQL Execution Messages.
+
+/*
+ * Represents a SQL query. Used in the command member of FlightDescriptor
+ * for the following RPC calls:
+ * - GetSchema: return the Arrow schema of the query.
+ * - GetFlightInfo: execute the query.
+ */
+message CommandStatementQuery {
+ option (experimental) = true;
+
+ // The SQL syntax.
+ string query = 1;
+}
+
+/**
+ * Represents a ticket resulting from GetFlightInfo with a CommandStatementQuery.
+ * This should be used only once and treated as an opaque value, that is, clients should not attempt to parse this.
+ */
+message TicketStatementQuery {
+ option (experimental) = true;
+
+ // Unique identifier for the instance of the statement to execute.
+ bytes statement_handle = 1;
+}
+
+/*
+ * Represents an instance of executing a prepared statement. Used in the command member of FlightDescriptor for
+ * the following RPC calls:
+ * - DoPut: bind parameter values. All of the bound parameter sets will be executed as a single atomic execution.
+ * - GetFlightInfo: execute the prepared statement instance.
+ */
+message CommandPreparedStatementQuery {
+ option (experimental) = true;
+
+ // Opaque handle for the prepared statement on the server.
+ bytes prepared_statement_handle = 1;
+}
+
+/*
+ * Represents a SQL update query. Used in the command member of FlightDescriptor
+ * for the the RPC call DoPut to cause the server to execute the included SQL update.
+ */
+message CommandStatementUpdate {
+ option (experimental) = true;
+
+ // The SQL syntax.
+ string query = 1;
+}
+
+/*
+ * Represents a SQL update query. Used in the command member of FlightDescriptor
+ * for the the RPC call DoPut to cause the server to execute the included
+ * prepared statement handle as an update.
+ */
+message CommandPreparedStatementUpdate {
+ option (experimental) = true;
+
+ // Opaque handle for the prepared statement on the server.
+ bytes prepared_statement_handle = 1;
+}
+
+/*
+ * Returned from the RPC call DoPut when a CommandStatementUpdate
+ * CommandPreparedStatementUpdate was in the request, containing
+ * results from the update.
+ */
+message DoPutUpdateResult {
+ option (experimental) = true;
+
+ // The number of records updated. A return value of -1 represents
+ // an unknown updated record count.
+ int64 record_count = 1;
+}
+
+extend google.protobuf.MessageOptions {
+ bool experimental = 1000;
+}
diff --git a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java
index 250b0edd2d323..a1bb8b667f4e8 100644
--- a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java
+++ b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java
@@ -17,18 +17,12 @@
package org.apache.arrow.adapter.jdbc;
-import static org.apache.arrow.vector.types.FloatingPointPrecision.DOUBLE;
-import static org.apache.arrow.vector.types.FloatingPointPrecision.SINGLE;
-
-import java.sql.Types;
import java.util.Calendar;
import java.util.Map;
import java.util.function.Function;
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.util.Preconditions;
-import org.apache.arrow.vector.types.DateUnit;
-import org.apache.arrow.vector.types.TimeUnit;
import org.apache.arrow.vector.types.pojo.ArrowType;
/**
@@ -55,16 +49,14 @@
*/
public final class JdbcToArrowConfig {
+ public static final int DEFAULT_TARGET_BATCH_SIZE = 1024;
+ public static final int NO_LIMIT_BATCH_SIZE = -1;
private final Calendar calendar;
private final BufferAllocator allocator;
private final boolean includeMetadata;
private final boolean reuseVectorSchemaRoot;
private final Map arraySubTypesByColumnIndex;
private final Map arraySubTypesByColumnName;
-
- public static final int DEFAULT_TARGET_BATCH_SIZE = 1024;
- public static final int NO_LIMIT_BATCH_SIZE = -1;
-
/**
* The maximum rowCount to read each time when partially convert data.
* Default value is 1024 and -1 means disable partial read.
@@ -82,7 +74,7 @@ public final class JdbcToArrowConfig {
/**
* Constructs a new configuration from the provided allocator and calendar. The allocator
* is used when constructing the Arrow vectors from the ResultSet, and the calendar is used to define
- * Arrow Timestamp fields, and to read time-based fields from the JDBC ResultSet.
+ * Arrow Timestamp fields, and to read time-based fields from the JDBC ResultSet.
*
* @param allocator The memory allocator to construct the Arrow vectors with.
* @param calendar The calendar to use when constructing Timestamp fields and reading time-based results.
@@ -99,7 +91,7 @@ public final class JdbcToArrowConfig {
/**
* Constructs a new configuration from the provided allocator and calendar. The allocator
* is used when constructing the Arrow vectors from the ResultSet, and the calendar is used to define
- * Arrow Timestamp fields, and to read time-based fields from the JDBC ResultSet.
+ * Arrow Timestamp fields, and to read time-based fields from the JDBC ResultSet.
*
* @param allocator The memory allocator to construct the Arrow vectors with.
* @param calendar The calendar to use when constructing Timestamp fields and reading time-based results.
@@ -134,6 +126,8 @@ public final class JdbcToArrowConfig {
*
*
*/
@@ -157,64 +151,7 @@ public final class JdbcToArrowConfig {
// set up type converter
this.jdbcToArrowTypeConverter = jdbcToArrowTypeConverter != null ? jdbcToArrowTypeConverter :
- fieldInfo -> {
- final String timezone;
- if (calendar != null) {
- timezone = calendar.getTimeZone().getID();
- } else {
- timezone = null;
- }
-
- 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;
- }
- };
+ jdbcFieldInfo -> JdbcToArrowUtils.getArrowTypeFromJdbcType(jdbcFieldInfo, calendar);
}
/**
@@ -230,6 +167,7 @@ public Calendar getCalendar() {
/**
* The Arrow memory allocator.
+ *
* @return the allocator.
*/
public BufferAllocator getAllocator() {
diff --git a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowUtils.java b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowUtils.java
index e05f21d48cf1a..db528af448630 100644
--- a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowUtils.java
+++ b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowUtils.java
@@ -17,13 +17,18 @@
package org.apache.arrow.adapter.jdbc;
+import static org.apache.arrow.vector.types.FloatingPointPrecision.DOUBLE;
+import static org.apache.arrow.vector.types.FloatingPointPrecision.SINGLE;
+
import java.io.IOException;
import java.sql.Date;
+import java.sql.ParameterMetaData;
import java.sql.ResultSet;
import java.sql.ResultSetMetaData;
import java.sql.SQLException;
import java.sql.Time;
import java.sql.Timestamp;
+import java.sql.Types;
import java.util.ArrayList;
import java.util.Calendar;
import java.util.HashMap;
@@ -70,6 +75,8 @@
import org.apache.arrow.vector.VarCharVector;
import org.apache.arrow.vector.VectorSchemaRoot;
import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.TimeUnit;
import org.apache.arrow.vector.types.pojo.ArrowType;
import org.apache.arrow.vector.types.pojo.Field;
import org.apache.arrow.vector.types.pojo.FieldType;
@@ -106,6 +113,101 @@ 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 parameterFields = new ArrayList<>(parameterMetaData.getParameterCount());
+ for (int parameterCounter = 1; parameterCounter <= parameterMetaData.getParameterCount();
+ parameterCounter++) {
+ final int jdbcDataType = parameterMetaData.getParameterType(parameterCounter);
+ final int jdbcIsNullable = parameterMetaData.isNullable(parameterCounter);
+ final boolean arrowIsNullable = jdbcIsNullable != ParameterMetaData.parameterNoNulls;
+ final int precision = parameterMetaData.getPrecision(parameterCounter);
+ final int scale = parameterMetaData.getScale(parameterCounter);
+ final ArrowType arrowType = getArrowTypeFromJdbcType(new JdbcFieldInfo(jdbcDataType, precision, scale), calendar);
+ final FieldType fieldType = new FieldType(arrowIsNullable, arrowType, /*dictionary=*/null);
+ parameterFields.add(new Field(null, fieldType, null));
+ }
+
+ return new Schema(parameterFields);
+ }
+
+ /**
+ * Converts the provided JDBC type to its respective {@link ArrowType} counterpart.
+ *
+ * @param fieldInfo the {@link JdbcFieldInfo} with information about the original JDBC type.
+ * @param calendar the {@link Calendar} to use for datetime data types.
+ * @return a new {@link ArrowType}.
+ */
+ public static ArrowType getArrowTypeFromJdbcType(final JdbcFieldInfo fieldInfo, final Calendar calendar) {
+ switch (fieldInfo.getJdbcType()) {
+ case Types.BOOLEAN:
+ case Types.BIT:
+ return new ArrowType.Bool();
+ case Types.TINYINT:
+ return new ArrowType.Int(8, true);
+ case Types.SMALLINT:
+ return new ArrowType.Int(16, true);
+ case Types.INTEGER:
+ return new ArrowType.Int(32, true);
+ case Types.BIGINT:
+ return new ArrowType.Int(64, true);
+ case Types.NUMERIC:
+ case Types.DECIMAL:
+ int precision = fieldInfo.getPrecision();
+ int scale = fieldInfo.getScale();
+ return new ArrowType.Decimal(precision, scale, 128);
+ case Types.REAL:
+ case Types.FLOAT:
+ return new ArrowType.FloatingPoint(SINGLE);
+ case Types.DOUBLE:
+ return new ArrowType.FloatingPoint(DOUBLE);
+ case Types.CHAR:
+ case Types.NCHAR:
+ case Types.VARCHAR:
+ case Types.NVARCHAR:
+ case Types.LONGVARCHAR:
+ case Types.LONGNVARCHAR:
+ case Types.CLOB:
+ return new ArrowType.Utf8();
+ case Types.DATE:
+ return new ArrowType.Date(DateUnit.DAY);
+ case Types.TIME:
+ return new ArrowType.Time(TimeUnit.MILLISECOND, 32);
+ case Types.TIMESTAMP:
+ final String timezone;
+ if (calendar != null) {
+ timezone = calendar.getTimeZone().getID();
+ } else {
+ timezone = null;
+ }
+ return new ArrowType.Timestamp(TimeUnit.MILLISECOND, timezone);
+ case Types.BINARY:
+ case Types.VARBINARY:
+ case Types.LONGVARBINARY:
+ case Types.BLOB:
+ return new ArrowType.Binary();
+ case Types.ARRAY:
+ return new ArrowType.List();
+ case Types.NULL:
+ return new ArrowType.Null();
+ case Types.STRUCT:
+ return new ArrowType.Struct();
+ default:
+ // no-op, shouldn't get here
+ return null;
+ }
+ }
+
/**
* Create Arrow {@link Schema} object for the given JDBC {@link java.sql.ResultSetMetaData}.
*
diff --git a/java/flight/flight-core/pom.xml b/java/flight/flight-core/pom.xml
index b1f00eb83f98e..c8ab5ac1d26d4 100644
--- a/java/flight/flight-core/pom.xml
+++ b/java/flight/flight-core/pom.xml
@@ -12,10 +12,10 @@
4.0.0
+ arrow-flightorg.apache.arrow
- arrow-java-root7.0.0-SNAPSHOT
- ../../pom.xml
+ ../pom.xmlflight-core
@@ -24,8 +24,6 @@
jar
- 1.41.0
- 3.7.11
diff --git a/java/flight/flight-grpc/pom.xml b/java/flight/flight-grpc/pom.xml
index c567b7cada5a4..a12e4e2665268 100644
--- a/java/flight/flight-grpc/pom.xml
+++ b/java/flight/flight-grpc/pom.xml
@@ -11,10 +11,10 @@
language governing permissions and limitations under the License. -->
- arrow-java-root
+ arrow-flightorg.apache.arrow7.0.0-SNAPSHOT
- ../../pom.xml
+ ../pom.xml4.0.0
@@ -24,8 +24,6 @@
jar
- 1.41.0
- 3.7.11
diff --git a/java/flight/flight-sql/pom.xml b/java/flight/flight-sql/pom.xml
new file mode 100644
index 0000000000000..b17ab9b7c48ae
--- /dev/null
+++ b/java/flight/flight-sql/pom.xml
@@ -0,0 +1,151 @@
+
+
+
+ 4.0.0
+
+ arrow-flight
+ org.apache.arrow
+ 7.0.0-SNAPSHOT
+ ../pom.xml
+
+
+ flight-sql
+ Arrow Flight SQL
+ (Experimental)Contains utility classes to expose Flight SQL semantics for clients and servers over Arrow Flight
+ jar
+
+
+ 1
+
+
+
+
+ org.apache.arrow
+ flight-core
+ ${project.version}
+
+
+ io.netty
+ netty-transport-native-unix-common
+
+
+ io.netty
+ netty-transport-native-kqueue
+
+
+ io.netty
+ netty-transport-native-epoll
+
+
+
+
+ org.apache.arrow
+ arrow-memory-core
+ ${project.version}
+
+
+ org.apache.arrow
+ arrow-jdbc
+ ${project.version}
+
+
+ io.grpc
+ grpc-protobuf
+ ${dep.grpc.version}
+
+
+ com.google.guava
+ guava
+
+
+ io.grpc
+ grpc-stub
+ ${dep.grpc.version}
+
+
+ com.google.protobuf
+ protobuf-java
+ ${dep.protobuf.version}
+
+
+ io.grpc
+ grpc-api
+ ${dep.grpc.version}
+
+
+ org.apache.arrow
+ arrow-vector
+ ${project.version}
+ ${arrow.vector.classifier}
+
+
+ org.slf4j
+ slf4j-api
+
+
+ org.apache.derby
+ derby
+ 10.14.2.0
+ test
+
+
+ org.apache.commons
+ commons-dbcp2
+ 2.9.0
+ test
+
+
+ commons-logging
+ commons-logging
+
+
+
+
+ org.apache.commons
+ commons-pool2
+ 2.11.1
+ test
+
+
+ org.hamcrest
+ hamcrest
+
+
+ commons-cli
+ commons-cli
+ 1.4
+
+
+
+
+
+
+ org.xolstice.maven.plugins
+ protobuf-maven-plugin
+
+
+ proto-compile
+ generate-sources
+
+ ${basedir}/../../../format/
+
+
+ compile
+ compile-custom
+
+
+
+
+
+
+
+
diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java
new file mode 100644
index 0000000000000..c1ff92a379621
--- /dev/null
+++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java
@@ -0,0 +1,631 @@
+/*
+ * 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.CommandGetCrossReference;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetDbSchemas;
+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.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.CommandPreparedStatementUpdate;
+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 static org.apache.arrow.flight.sql.impl.FlightSql.SqlInfo;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.channels.Channels;
+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.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.ActionCreatePreparedStatementResult;
+import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery;
+import org.apache.arrow.flight.sql.util.TableRef;
+import org.apache.arrow.memory.ArrowBuf;
+import org.apache.arrow.util.AutoCloseables;
+import org.apache.arrow.util.Preconditions;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.ipc.ReadChannel;
+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;
+
+/**
+ * Flight client with Flight SQL semantics.
+ */
+public class FlightSqlClient implements AutoCloseable {
+ 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 dbSchemaFilterPattern 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 dbSchemaFilterPattern, final CallOption... options) {
+ final CommandGetDbSchemas.Builder builder = CommandGetDbSchemas.newBuilder();
+
+ if (catalog != null) {
+ builder.setCatalog(catalog);
+ }
+
+ if (dbSchemaFilterPattern != null) {
+ builder.setDbSchemaFilterPattern(dbSchemaFilterPattern);
+ }
+
+ 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.
+ *
+ * @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 SqlInfo... 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 SqlInfo[] info, final CallOption... options) {
+ final int[] infoNumbers = Arrays.stream(info).mapToInt(SqlInfo::getNumber).toArray();
+ return getSqlInfo(infoNumbers, options);
+ }
+
+ /**
+ * Request a set of Flight SQL metadata.
+ * Use this method if you would like to retrieve custom metadata, where the custom metadata key values start
+ * from 10_000.
+ *
+ * @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.
+ * Use this method if you would like to retrieve custom metadata, where the custom metadata key values start
+ * from 10_000.
+ *
+ * @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 Iterable 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 dbSchemaFilterPattern 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 dbSchemaFilterPattern,
+ final String tableFilterPattern, final List tableTypes,
+ final boolean includeSchema, final CallOption... options) {
+ final CommandGetTables.Builder builder = CommandGetTables.newBuilder();
+
+ if (catalog != null) {
+ builder.setCatalog(catalog);
+ }
+
+ if (dbSchemaFilterPattern != null) {
+ builder.setDbSchemaFilterPattern(dbSchemaFilterPattern);
+ }
+
+ if (tableFilterPattern != null) {
+ builder.setTableNameFilterPattern(tableFilterPattern);
+ }
+
+ 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 tableRef An object which hold info about catalog, dbSchema and table.
+ * @param options RPC-layer hints for this call.
+ * @return a FlightInfo object representing the stream(s) to fetch.
+ */
+ public FlightInfo getPrimaryKeys(final TableRef tableRef, final CallOption... options) {
+ final CommandGetPrimaryKeys.Builder builder = CommandGetPrimaryKeys.newBuilder();
+
+ if (tableRef.getCatalog() != null) {
+ builder.setCatalog(tableRef.getCatalog());
+ }
+
+ if (tableRef.getDbSchema() != null) {
+ builder.setDbSchema(tableRef.getDbSchema());
+ }
+
+ Objects.requireNonNull(tableRef.getTable());
+ builder.setTable(tableRef.getTable()).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 tableRef An object which hold info about catalog, dbSchema and table.
+ * @param options RPC-layer hints for this call.
+ * @return a FlightInfo object representing the stream(s) to fetch.
+ */
+ public FlightInfo getExportedKeys(final TableRef tableRef, final CallOption... options) {
+ Objects.requireNonNull(tableRef.getTable(), "Table cannot be null.");
+
+ final CommandGetExportedKeys.Builder builder = CommandGetExportedKeys.newBuilder();
+
+ if (tableRef.getCatalog() != null) {
+ builder.setCatalog(tableRef.getCatalog());
+ }
+
+ if (tableRef.getDbSchema() != null) {
+ builder.setDbSchema(tableRef.getDbSchema());
+ }
+
+ Objects.requireNonNull(tableRef.getTable());
+ builder.setTable(tableRef.getTable()).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 tableRef An object which hold info about catalog, dbSchema and table.
+ * @param options RPC-layer hints for this call.
+ * @return a FlightInfo object representing the stream(s) to fetch.
+ */
+ public FlightInfo getImportedKeys(final TableRef tableRef,
+ final CallOption... options) {
+ Objects.requireNonNull(tableRef.getTable(), "Table cannot be null.");
+
+ final CommandGetImportedKeys.Builder builder = CommandGetImportedKeys.newBuilder();
+
+ if (tableRef.getCatalog() != null) {
+ builder.setCatalog(tableRef.getCatalog());
+ }
+
+ if (tableRef.getDbSchema() != null) {
+ builder.setDbSchema(tableRef.getDbSchema());
+ }
+
+ Objects.requireNonNull(tableRef.getTable());
+ builder.setTable(tableRef.getTable()).build();
+
+ final FlightDescriptor descriptor = FlightDescriptor.command(Any.pack(builder.build()).toByteArray());
+ return client.getInfo(descriptor, options);
+ }
+
+ /**
+ * Retrieves a description of the foreign key columns that reference the given table's
+ * primary key columns (the foreign keys exported by a table).
+ *
+ * @param pkTableRef An object which hold info about catalog, dbSchema and table from a primary table.
+ * @param fkTableRef An object which hold info about catalog, dbSchema and table from a foreign table.
+ * @param options RPC-layer hints for this call.
+ * @return a FlightInfo object representing the stream(s) to fetch.
+ */
+ public FlightInfo getCrossReference(final TableRef pkTableRef,
+ final TableRef fkTableRef, final CallOption... options) {
+ Objects.requireNonNull(pkTableRef.getTable(), "Parent Table cannot be null.");
+ Objects.requireNonNull(fkTableRef.getTable(), "Foreign Table cannot be null.");
+
+ final CommandGetCrossReference.Builder builder = CommandGetCrossReference.newBuilder();
+
+ if (pkTableRef.getCatalog() != null) {
+ builder.setPkCatalog(pkTableRef.getCatalog());
+ }
+
+ if (pkTableRef.getDbSchema() != null) {
+ builder.setPkDbSchema(pkTableRef.getDbSchema());
+ }
+
+ if (fkTableRef.getCatalog() != null) {
+ builder.setFkCatalog(fkTableRef.getCatalog());
+ }
+
+ if (fkTableRef.getDbSchema() != null) {
+ builder.setFkDbSchema(fkTableRef.getDbSchema());
+ }
+
+ builder.setPkTable(pkTableRef.getTable());
+ builder.setFkTable(fkTableRef.getTable());
+
+ 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);
+ }
+
+ @Override
+ public void close() throws SQLException {
+ try {
+ AutoCloseables.close(client);
+ } catch (final Exception e) {
+ throw new SQLException(e);
+ }
+ }
+
+ /**
+ * 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_CREATE_PREPARED_STATEMENT.getType(),
+ Any.pack(ActionCreatePreparedStatementRequest
+ .newBuilder()
+ .setQuery(sql)
+ .build())
+ .toByteArray());
+ final Iterator 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 = parameterBindingRoot;
+ }
+
+ /**
+ * Closes the {@link #parameterBindingRoot}, which contains the parameter binding from
+ * a {@link PreparedStatement} operation, releasing its resources.
+ */
+ public void clearParameters() {
+ if (parameterBindingRoot != null) {
+ parameterBindingRoot.close();
+ }
+ }
+
+ /**
+ * Returns the Schema of the resultset.
+ *
+ * @return the Schema of the resultset.
+ */
+ public Schema getResultSetSchema() {
+ if (resultSetSchema == null) {
+ final ByteString bytes = preparedStatementResult.getDatasetSchema();
+ resultSetSchema = deserializeSchema(bytes);
+ }
+ return resultSetSchema;
+ }
+
+ /**
+ * Returns the Schema of the parameters.
+ *
+ * @return the Schema of the parameters.
+ */
+ public Schema getParameterSchema() {
+ if (parameterSchema == null) {
+ final ByteString bytes = preparedStatementResult.getParameterSchema();
+ parameterSchema = deserializeSchema(bytes);
+ }
+ return parameterSchema;
+ }
+
+ private Schema deserializeSchema(final ByteString bytes) {
+ try {
+ return bytes.isEmpty() ?
+ new Schema(Collections.emptyList()) :
+ MessageSerializer.deserializeSchema(
+ new ReadChannel(Channels.newChannel(
+ new ByteArrayInputStream(bytes.toByteArray()))));
+ } catch (final IOException e) {
+ throw new RuntimeException("Failed to deserialize schema", e);
+ }
+ }
+
+ /**
+ * Executes the prepared statement query on the server.
+ *
+ * @param options RPC-layer hints for this call.
+ * @return a FlightInfo object representing the stream(s) to fetch.
+ */
+ public FlightInfo execute(final CallOption... options) throws SQLException {
+ checkOpen();
+
+ final FlightDescriptor descriptor = FlightDescriptor
+ .command(Any.pack(CommandPreparedStatementQuery.newBuilder()
+ .setPreparedStatementHandle(preparedStatementResult.getPreparedStatementHandle())
+ .build())
+ .toByteArray());
+
+ if (parameterBindingRoot != null && parameterBindingRoot.getRowCount() > 0) {
+ final SyncPutListener putListener = new SyncPutListener();
+
+ FlightClient.ClientStreamListener listener =
+ client.startPut(descriptor, parameterBindingRoot, putListener, options);
+
+ listener.putNext();
+ listener.completed();
+ }
+
+ return client.getInfo(descriptor, options);
+ }
+
+ /**
+ * Checks whether this client is open.
+ *
+ * @throws IllegalStateException if client is closed.
+ */
+ protected final void checkOpen() {
+ Preconditions.checkState(!isClosed, "Statement closed");
+ }
+
+ /**
+ * Executes the prepared statement update on the server.
+ *
+ * @param options RPC-layer hints for this call.
+ * @return the count of updated records
+ */
+ public long executeUpdate(final CallOption... options) {
+ checkOpen();
+ final FlightDescriptor descriptor = FlightDescriptor
+ .command(Any.pack(CommandPreparedStatementUpdate.newBuilder()
+ .setPreparedStatementHandle(preparedStatementResult.getPreparedStatementHandle())
+ .build())
+ .toByteArray());
+ setParameters(parameterBindingRoot == null ? VectorSchemaRoot.of() : parameterBindingRoot);
+ final SyncPutListener putListener = new SyncPutListener();
+ final FlightClient.ClientStreamListener listener =
+ client.startPut(descriptor, parameterBindingRoot, putListener, options);
+ listener.putNext();
+ listener.completed();
+ 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.INVALID_ARGUMENT.withCause(e).toRuntimeException();
+ }
+ }
+
+ /**
+ * Closes the client.
+ *
+ * @param options RPC-layer hints for this call.
+ */
+ public void close(final CallOption... options) {
+ if (isClosed) {
+ return;
+ }
+ isClosed = true;
+ final Action action = new Action(
+ FlightSqlUtils.FLIGHT_SQL_CLOSE_PREPARED_STATEMENT.getType(),
+ Any.pack(ActionClosePreparedStatementRequest.newBuilder()
+ .setPreparedStatementHandle(preparedStatementResult.getPreparedStatementHandle())
+ .build())
+ .toByteArray());
+ final Iterator closePreparedStatementResults = client.doAction(action, options);
+ closePreparedStatementResults.forEachRemaining(result -> {
+ });
+ if (parameterBindingRoot != null) {
+ parameterBindingRoot.close();
+ }
+ }
+
+ @Override
+ public void close() {
+ close(new CallOption[0]);
+ }
+
+ /**
+ * Returns if the prepared statement is already closed.
+ *
+ * @return true if the prepared statement is already closed.
+ */
+ public boolean isClosed() {
+ return isClosed;
+ }
+ }
+}
diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java
new file mode 100644
index 0000000000000..87c8b3e092dba
--- /dev/null
+++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java
@@ -0,0 +1,669 @@
+/*
+ * 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 java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static java.util.stream.IntStream.range;
+import static org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementResult;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetCrossReference;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetDbSchemas;
+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.vector.complex.MapVector.DATA_VECTOR_NAME;
+import static org.apache.arrow.vector.complex.MapVector.KEY_NAME;
+import static org.apache.arrow.vector.complex.MapVector.VALUE_NAME;
+import static org.apache.arrow.vector.types.Types.MinorType.BIGINT;
+import static org.apache.arrow.vector.types.Types.MinorType.BIT;
+import static org.apache.arrow.vector.types.Types.MinorType.INT;
+import static org.apache.arrow.vector.types.Types.MinorType.LIST;
+import static org.apache.arrow.vector.types.Types.MinorType.STRUCT;
+import static org.apache.arrow.vector.types.Types.MinorType.UINT4;
+import static org.apache.arrow.vector.types.Types.MinorType.VARCHAR;
+
+import java.util.List;
+
+import org.apache.arrow.flight.Action;
+import org.apache.arrow.flight.ActionType;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightDescriptor;
+import org.apache.arrow.flight.FlightInfo;
+import org.apache.arrow.flight.FlightProducer;
+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.Ticket;
+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.CommandGetCatalogs;
+import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetPrimaryKeys;
+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.flight.sql.impl.FlightSql.DoPutUpdateResult;
+import org.apache.arrow.flight.sql.impl.FlightSql.TicketStatementQuery;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.UnionMode;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.ArrowType.Union;
+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 com.google.common.collect.ImmutableList;
+import com.google.protobuf.Any;
+import com.google.protobuf.InvalidProtocolBufferException;
+
+/**
+ * API to Implement an Arrow Flight SQL producer.
+ */
+public interface FlightSqlProducer extends FlightProducer, AutoCloseable {
+ /**
+ * Depending on the provided command, method either:
+ * 1. Return information about a SQL query, or
+ * 2. Return information about a prepared statement. In this case, parameters binding is allowed.
+ *
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return information about the given SQL query, or the given prepared statement.
+ */
+ @Override
+ default FlightInfo getFlightInfo(CallContext context, FlightDescriptor descriptor) {
+ final Any command = FlightSqlUtils.parseOrThrow(descriptor.getCommand());
+
+ if (command.is(CommandStatementQuery.class)) {
+ return getFlightInfoStatement(
+ FlightSqlUtils.unpackOrThrow(command, CommandStatementQuery.class), context, descriptor);
+ } else if (command.is(CommandPreparedStatementQuery.class)) {
+ return getFlightInfoPreparedStatement(
+ FlightSqlUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), context, descriptor);
+ } else if (command.is(CommandGetCatalogs.class)) {
+ return getFlightInfoCatalogs(
+ FlightSqlUtils.unpackOrThrow(command, CommandGetCatalogs.class), context, descriptor);
+ } else if (command.is(CommandGetDbSchemas.class)) {
+ return getFlightInfoSchemas(
+ FlightSqlUtils.unpackOrThrow(command, CommandGetDbSchemas.class), context, descriptor);
+ } else if (command.is(CommandGetTables.class)) {
+ return getFlightInfoTables(
+ FlightSqlUtils.unpackOrThrow(command, CommandGetTables.class), context, descriptor);
+ } else if (command.is(CommandGetTableTypes.class)) {
+ return getFlightInfoTableTypes(
+ FlightSqlUtils.unpackOrThrow(command, CommandGetTableTypes.class), context, descriptor);
+ } else if (command.is(CommandGetSqlInfo.class)) {
+ return getFlightInfoSqlInfo(
+ FlightSqlUtils.unpackOrThrow(command, CommandGetSqlInfo.class), context, descriptor);
+ } else if (command.is(CommandGetPrimaryKeys.class)) {
+ return getFlightInfoPrimaryKeys(
+ FlightSqlUtils.unpackOrThrow(command, CommandGetPrimaryKeys.class), context, descriptor);
+ } else if (command.is(CommandGetExportedKeys.class)) {
+ return getFlightInfoExportedKeys(
+ FlightSqlUtils.unpackOrThrow(command, CommandGetExportedKeys.class), context, descriptor);
+ } else if (command.is(CommandGetImportedKeys.class)) {
+ return getFlightInfoImportedKeys(
+ FlightSqlUtils.unpackOrThrow(command, CommandGetImportedKeys.class), context, descriptor);
+ } else if (command.is(CommandGetCrossReference.class)) {
+ return getFlightInfoCrossReference(
+ FlightSqlUtils.unpackOrThrow(command, CommandGetCrossReference.class), context, descriptor);
+ }
+
+ throw CallStatus.INVALID_ARGUMENT.withDescription("The defined request is invalid.").toRuntimeException();
+ }
+
+ /**
+ * Returns the schema of the result produced by the SQL query.
+ *
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return the result set schema.
+ */
+ @Override
+ default SchemaResult getSchema(CallContext context, FlightDescriptor descriptor) {
+ final Any command = FlightSqlUtils.parseOrThrow(descriptor.getCommand());
+
+ if (command.is(CommandStatementQuery.class)) {
+ return getSchemaStatement(
+ FlightSqlUtils.unpackOrThrow(command, CommandStatementQuery.class), context, descriptor);
+ } else if (command.is(CommandGetCatalogs.class)) {
+ return new SchemaResult(Schemas.GET_CATALOGS_SCHEMA);
+ } else if (command.is(CommandGetDbSchemas.class)) {
+ return new SchemaResult(Schemas.GET_SCHEMAS_SCHEMA);
+ } else if (command.is(CommandGetTables.class)) {
+ return new SchemaResult(Schemas.GET_TABLES_SCHEMA);
+ } else if (command.is(CommandGetTableTypes.class)) {
+ return new SchemaResult(Schemas.GET_TABLE_TYPES_SCHEMA);
+ } else if (command.is(CommandGetSqlInfo.class)) {
+ return new SchemaResult(Schemas.GET_SQL_INFO_SCHEMA);
+ } else if (command.is(CommandGetPrimaryKeys.class)) {
+ return new SchemaResult(Schemas.GET_PRIMARY_KEYS_SCHEMA);
+ } else if (command.is(CommandGetImportedKeys.class)) {
+ return new SchemaResult(Schemas.GET_IMPORTED_KEYS_SCHEMA);
+ } else if (command.is(CommandGetExportedKeys.class)) {
+ return new SchemaResult(Schemas.GET_EXPORTED_KEYS_SCHEMA);
+ } else if (command.is(CommandGetCrossReference.class)) {
+ return new SchemaResult(Schemas.GET_CROSS_REFERENCE_SCHEMA);
+ }
+
+ throw CallStatus.INVALID_ARGUMENT.withDescription("Invalid command provided.").toRuntimeException();
+ }
+
+ /**
+ * Depending on the provided command, method either:
+ * 1. Return data for a stream produced by executing the provided SQL query, or
+ * 2. Return data for a prepared statement. In this case, parameters binding is allowed.
+ *
+ * @param context Per-call context.
+ * @param ticket The application-defined ticket identifying this stream.
+ * @param listener An interface for sending data back to the client.
+ */
+ @Override
+ default void getStream(CallContext context, Ticket ticket, ServerStreamListener listener) {
+ final Any command;
+
+ try {
+ command = Any.parseFrom(ticket.getBytes());
+ } catch (InvalidProtocolBufferException e) {
+ listener.error(e);
+ return;
+ }
+
+ if (command.is(TicketStatementQuery.class)) {
+ getStreamStatement(
+ FlightSqlUtils.unpackOrThrow(command, TicketStatementQuery.class), context, listener);
+ } else if (command.is(CommandPreparedStatementQuery.class)) {
+ getStreamPreparedStatement(
+ FlightSqlUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), context, listener);
+ } else if (command.is(CommandGetCatalogs.class)) {
+ getStreamCatalogs(context, listener);
+ } else if (command.is(CommandGetDbSchemas.class)) {
+ getStreamSchemas(FlightSqlUtils.unpackOrThrow(command, CommandGetDbSchemas.class), context, listener);
+ } else if (command.is(CommandGetTables.class)) {
+ getStreamTables(FlightSqlUtils.unpackOrThrow(command, CommandGetTables.class), context, listener);
+ } else if (command.is(CommandGetTableTypes.class)) {
+ getStreamTableTypes(context, listener);
+ } else if (command.is(CommandGetSqlInfo.class)) {
+ getStreamSqlInfo(FlightSqlUtils.unpackOrThrow(command, CommandGetSqlInfo.class), context, listener);
+ } else if (command.is(CommandGetPrimaryKeys.class)) {
+ getStreamPrimaryKeys(FlightSqlUtils.unpackOrThrow(command, CommandGetPrimaryKeys.class), context, listener);
+ } else if (command.is(CommandGetExportedKeys.class)) {
+ getStreamExportedKeys(FlightSqlUtils.unpackOrThrow(command, CommandGetExportedKeys.class), context, listener);
+ } else if (command.is(CommandGetImportedKeys.class)) {
+ getStreamImportedKeys(FlightSqlUtils.unpackOrThrow(command, CommandGetImportedKeys.class), context, listener);
+ } else if (command.is(CommandGetCrossReference.class)) {
+ getStreamCrossReference(FlightSqlUtils.unpackOrThrow(command, CommandGetCrossReference.class), context, listener);
+ } else {
+ throw CallStatus.INVALID_ARGUMENT.withDescription("The defined request is invalid.").toRuntimeException();
+ }
+ }
+
+ /**
+ * Depending on the provided command, method either:
+ * 1. Execute provided SQL query as an update statement, or
+ * 2. Execute provided update SQL query prepared statement. In this case, parameters binding
+ * is allowed, or
+ * 3. Binds parameters to the provided prepared statement.
+ *
+ * @param context Per-call context.
+ * @param flightStream The data stream being uploaded.
+ * @param ackStream The data stream listener for update result acknowledgement.
+ * @return a Runnable to process the stream.
+ */
+ @Override
+ default Runnable acceptPut(CallContext context, FlightStream flightStream, StreamListener ackStream) {
+ final Any command = FlightSqlUtils.parseOrThrow(flightStream.getDescriptor().getCommand());
+
+ if (command.is(CommandStatementUpdate.class)) {
+ return acceptPutStatement(
+ FlightSqlUtils.unpackOrThrow(command, CommandStatementUpdate.class),
+ context, flightStream, ackStream);
+ } else if (command.is(CommandPreparedStatementUpdate.class)) {
+ return acceptPutPreparedStatementUpdate(
+ FlightSqlUtils.unpackOrThrow(command, CommandPreparedStatementUpdate.class),
+ context, flightStream, ackStream);
+ } else if (command.is(CommandPreparedStatementQuery.class)) {
+ return acceptPutPreparedStatementQuery(
+ FlightSqlUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class),
+ context, flightStream, ackStream);
+ }
+
+ throw CallStatus.INVALID_ARGUMENT.withDescription("The defined request is invalid.").toRuntimeException();
+ }
+
+ /**
+ * Lists all available Flight SQL actions.
+ *
+ * @param context Per-call context.
+ * @param listener An interface for sending data back to the client.
+ */
+ @Override
+ default void listActions(CallContext context, StreamListener listener) {
+ FlightSqlUtils.FLIGHT_SQL_ACTIONS.forEach(listener::onNext);
+ listener.onCompleted();
+ }
+
+ /**
+ * Performs the requested Flight SQL action.
+ *
+ * @param context Per-call context.
+ * @param action Client-supplied parameters.
+ * @param listener A stream of responses.
+ */
+ @Override
+ default void doAction(CallContext context, Action action, StreamListener listener) {
+ final String actionType = action.getType();
+ if (actionType.equals(FlightSqlUtils.FLIGHT_SQL_CREATE_PREPARED_STATEMENT.getType())) {
+ final ActionCreatePreparedStatementRequest request = FlightSqlUtils.unpackAndParseOrThrow(action.getBody(),
+ ActionCreatePreparedStatementRequest.class);
+ createPreparedStatement(request, context, listener);
+ } else if (actionType.equals(FlightSqlUtils.FLIGHT_SQL_CLOSE_PREPARED_STATEMENT.getType())) {
+ final ActionClosePreparedStatementRequest request = FlightSqlUtils.unpackAndParseOrThrow(action.getBody(),
+ ActionClosePreparedStatementRequest.class);
+ closePreparedStatement(request, context, listener);
+ }
+
+ throw CallStatus.INVALID_ARGUMENT.withDescription("Invalid action provided.").toRuntimeException();
+ }
+
+ /**
+ * Creates a prepared statement on the server and returns a handle and metadata for in a
+ * {@link ActionCreatePreparedStatementResult} object in a {@link Result}
+ * object.
+ *
+ * @param request The sql command to generate the prepared statement.
+ * @param context Per-call context.
+ * @param listener A stream of responses.
+ */
+ void createPreparedStatement(ActionCreatePreparedStatementRequest request, CallContext context,
+ StreamListener listener);
+
+ /**
+ * Closes a prepared statement on the server. No result is expected.
+ *
+ * @param request The sql command to generate the prepared statement.
+ * @param context Per-call context.
+ * @param listener A stream of responses.
+ */
+ void closePreparedStatement(ActionClosePreparedStatementRequest request, CallContext context,
+ StreamListener listener);
+
+ /**
+ * Gets information about a particular SQL query based data stream.
+ *
+ * @param command The sql command to generate the data stream.
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return Metadata about the stream.
+ */
+ FlightInfo getFlightInfoStatement(CommandStatementQuery command, CallContext context,
+ FlightDescriptor descriptor);
+
+ /**
+ * Gets information about a particular prepared statement data stream.
+ *
+ * @param command The prepared statement to generate the data stream.
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return Metadata about the stream.
+ */
+ FlightInfo getFlightInfoPreparedStatement(CommandPreparedStatementQuery command,
+ CallContext context, FlightDescriptor descriptor);
+
+ /**
+ * Gets schema about a particular SQL query based data stream.
+ *
+ * @param command The sql command to generate the data stream.
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return Schema for the stream.
+ */
+ SchemaResult getSchemaStatement(CommandStatementQuery command, CallContext context,
+ FlightDescriptor descriptor);
+
+ /**
+ * Returns data for a SQL query based data stream.
+ * @param ticket Ticket message containing the statement handle.
+ * @param context Per-call context.
+ * @param listener An interface for sending data back to the client.
+ */
+ void getStreamStatement(TicketStatementQuery ticket, CallContext context,
+ ServerStreamListener listener);
+
+ /**
+ * Returns data for a particular prepared statement query instance.
+ *
+ * @param command The prepared statement to generate the data stream.
+ * @param context Per-call context.
+ * @param listener An interface for sending data back to the client.
+ */
+ void getStreamPreparedStatement(CommandPreparedStatementQuery command, CallContext context,
+ ServerStreamListener listener);
+
+ /**
+ * Accepts uploaded data for a particular SQL query based data stream.
+ *
`PutResult`s must be in the form of a {@link DoPutUpdateResult}.
+ *
+ * @param command The sql command to generate the data stream.
+ * @param context Per-call context.
+ * @param flightStream The data stream being uploaded.
+ * @param ackStream The result data stream.
+ * @return A runnable to process the stream.
+ */
+ Runnable acceptPutStatement(CommandStatementUpdate command, CallContext context,
+ FlightStream flightStream, StreamListener ackStream);
+
+ /**
+ * Accepts uploaded data for a particular prepared statement data stream.
+ *
`PutResult`s must be in the form of a {@link DoPutUpdateResult}.
+ *
+ * @param command The prepared statement to generate the data stream.
+ * @param context Per-call context.
+ * @param flightStream The data stream being uploaded.
+ * @param ackStream The result data stream.
+ * @return A runnable to process the stream.
+ */
+ Runnable acceptPutPreparedStatementUpdate(CommandPreparedStatementUpdate command,
+ CallContext context, FlightStream flightStream,
+ StreamListener ackStream);
+
+ /**
+ * Accepts uploaded parameter values for a particular prepared statement query.
+ *
+ * @param command The prepared statement the parameter values will bind to.
+ * @param context Per-call context.
+ * @param flightStream The data stream being uploaded.
+ * @param ackStream The result data stream.
+ * @return A runnable to process the stream.
+ */
+ Runnable acceptPutPreparedStatementQuery(CommandPreparedStatementQuery command,
+ CallContext context, FlightStream flightStream,
+ StreamListener ackStream);
+
+ /**
+ * Returns the SQL Info of the server by returning a
+ * {@link CommandGetSqlInfo} in a {@link Result}.
+ *
+ * @param request request filter parameters.
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return Metadata about the stream.
+ */
+ FlightInfo getFlightInfoSqlInfo(CommandGetSqlInfo request, CallContext context,
+ FlightDescriptor descriptor);
+
+ /**
+ * Returns data for SQL info based data stream.
+ *
+ * @param command The command to generate the data stream.
+ * @param context Per-call context.
+ * @param listener An interface for sending data back to the client.
+ */
+ void getStreamSqlInfo(CommandGetSqlInfo command, CallContext context, ServerStreamListener listener);
+
+ /**
+ * Returns the available catalogs by returning a stream of
+ * {@link CommandGetCatalogs} objects in {@link Result} objects.
+ *
+ * @param request request filter parameters.
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return Metadata about the stream.
+ */
+ FlightInfo getFlightInfoCatalogs(CommandGetCatalogs request, CallContext context,
+ FlightDescriptor descriptor);
+
+ /**
+ * Returns data for catalogs based data stream.
+ *
+ * @param context Per-call context.
+ * @param listener An interface for sending data back to the client.
+ */
+ void getStreamCatalogs(CallContext context, ServerStreamListener listener);
+
+ /**
+ * Returns the available schemas by returning a stream of
+ * {@link CommandGetDbSchemas} objects in {@link Result} objects.
+ *
+ * @param request request filter parameters.
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return Metadata about the stream.
+ */
+ FlightInfo getFlightInfoSchemas(CommandGetDbSchemas request, CallContext context,
+ FlightDescriptor descriptor);
+
+ /**
+ * Returns data for schemas based data stream.
+ *
+ * @param command The command to generate the data stream.
+ * @param context Per-call context.
+ * @param listener An interface for sending data back to the client.
+ */
+ void getStreamSchemas(CommandGetDbSchemas command, CallContext context, ServerStreamListener listener);
+
+ /**
+ * Returns the available tables by returning a stream of
+ * {@link CommandGetTables} objects in {@link Result} objects.
+ *
+ * @param request request filter parameters.
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return Metadata about the stream.
+ */
+ FlightInfo getFlightInfoTables(CommandGetTables request, CallContext context,
+ FlightDescriptor descriptor);
+
+ /**
+ * Returns data for tables based data stream.
+ *
+ * @param command The command to generate the data stream.
+ * @param context Per-call context.
+ * @param listener An interface for sending data back to the client.
+ */
+ void getStreamTables(CommandGetTables command, CallContext context, ServerStreamListener listener);
+
+ /**
+ * Returns the available table types by returning a stream of
+ * {@link CommandGetTableTypes} objects in {@link Result} objects.
+ *
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return Metadata about the stream.
+ */
+ FlightInfo getFlightInfoTableTypes(CommandGetTableTypes request, CallContext context,
+ FlightDescriptor descriptor);
+
+ /**
+ * Returns data for table types based data stream.
+ *
+ * @param context Per-call context.
+ * @param listener An interface for sending data back to the client.
+ */
+ void getStreamTableTypes(CallContext context, ServerStreamListener listener);
+
+ /**
+ * Returns the available primary keys by returning a stream of
+ * {@link CommandGetPrimaryKeys} objects in {@link Result} objects.
+ *
+ * @param request request filter parameters.
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return Metadata about the stream.
+ */
+ FlightInfo getFlightInfoPrimaryKeys(CommandGetPrimaryKeys request, CallContext context,
+ FlightDescriptor descriptor);
+
+ /**
+ * Returns data for primary keys based data stream.
+ *
+ * @param command The command to generate the data stream.
+ * @param context Per-call context.
+ * @param listener An interface for sending data back to the client.
+ */
+ void getStreamPrimaryKeys(CommandGetPrimaryKeys command, CallContext context,
+ ServerStreamListener listener);
+
+ /**
+ * Retrieves a description of the foreign key columns that reference the given table's primary key columns
+ * {@link CommandGetExportedKeys} objects in {@link Result} objects.
+ *
+ * @param request request filter parameters.
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return Metadata about the stream.
+ */
+ FlightInfo getFlightInfoExportedKeys(CommandGetExportedKeys request, CallContext context,
+ FlightDescriptor descriptor);
+
+ /**
+ * Retrieves a description of the primary key columns that are referenced by given table's foreign key columns
+ * {@link CommandGetImportedKeys} objects in {@link Result} objects.
+ *
+ * @param request request filter parameters.
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return Metadata about the stream.
+ */
+ FlightInfo getFlightInfoImportedKeys(CommandGetImportedKeys request, CallContext context,
+ FlightDescriptor descriptor);
+
+ /**
+ * Retrieve a description of the foreign key columns that reference the given table's primary key columns
+ * {@link CommandGetCrossReference} objects in {@link Result} objects.
+ *
+ * @param request request filter parameters.
+ * @param context Per-call context.
+ * @param descriptor The descriptor identifying the data stream.
+ * @return Metadata about the stream.
+ */
+ FlightInfo getFlightInfoCrossReference(CommandGetCrossReference request, CallContext context,
+ FlightDescriptor descriptor);
+
+ /**
+ * Returns data for foreign keys based data stream.
+ *
+ * @param command The command to generate the data stream.
+ * @param context Per-call context.
+ * @param listener An interface for sending data back to the client.
+ */
+ void getStreamExportedKeys(CommandGetExportedKeys command, CallContext context,
+ ServerStreamListener listener);
+
+ /**
+ * Returns data for foreign keys based data stream.
+ *
+ * @param command The command to generate the data stream.
+ * @param context Per-call context.
+ * @param listener An interface for sending data back to the client.
+ */
+ void getStreamImportedKeys(CommandGetImportedKeys command, CallContext context,
+ ServerStreamListener listener);
+
+ /**
+ * Returns data for cross reference based data stream.
+ *
+ * @param command The command to generate the data stream.
+ * @param context Per-call context.
+ * @param listener An interface for sending data back to the client.
+ */
+ void getStreamCrossReference(CommandGetCrossReference command, CallContext context,
+ ServerStreamListener listener);
+
+
+ /**
+ * Default schema templates for the {@link FlightSqlProducer}.
+ */
+ final class Schemas {
+ public static final Schema GET_TABLES_SCHEMA = new Schema(asList(
+ Field.nullable("catalog_name", VARCHAR.getType()),
+ Field.nullable("schema_name", VARCHAR.getType()),
+ Field.notNullable("table_name", VARCHAR.getType()),
+ Field.notNullable("table_type", VARCHAR.getType()),
+ Field.notNullable("table_schema", MinorType.VARBINARY.getType())));
+ public static final Schema GET_TABLES_SCHEMA_NO_SCHEMA = new Schema(asList(
+ Field.nullable("catalog_name", VARCHAR.getType()),
+ Field.nullable("schema_name", VARCHAR.getType()),
+ Field.notNullable("table_name", VARCHAR.getType()),
+ Field.notNullable("table_type", VARCHAR.getType())));
+ public static final Schema GET_CATALOGS_SCHEMA = new Schema(
+ singletonList(Field.notNullable("catalog_name", VARCHAR.getType())));
+ public static final Schema GET_TABLE_TYPES_SCHEMA =
+ new Schema(singletonList(Field.notNullable("table_type", VARCHAR.getType())));
+ public static final Schema GET_SCHEMAS_SCHEMA =
+ new Schema(asList(
+ Field.nullable("catalog_name", VARCHAR.getType()),
+ Field.notNullable("schema_name", VARCHAR.getType())));
+ private static final Schema GET_IMPORTED_EXPORTED_AND_CROSS_REFERENCE_KEYS_SCHEMA =
+ new Schema(asList(
+ Field.nullable("pk_catalog_name", VARCHAR.getType()),
+ Field.nullable("pk_schema_name", VARCHAR.getType()),
+ Field.notNullable("pk_table_name", VARCHAR.getType()),
+ Field.notNullable("pk_column_name", VARCHAR.getType()),
+ Field.nullable("fk_catalog_name", VARCHAR.getType()),
+ Field.nullable("fk_schema_name", VARCHAR.getType()),
+ Field.notNullable("fk_table_name", VARCHAR.getType()),
+ Field.notNullable("fk_column_name", VARCHAR.getType()),
+ Field.notNullable("key_sequence", INT.getType()),
+ Field.nullable("fk_key_name", VARCHAR.getType()),
+ Field.nullable("pk_key_name", VARCHAR.getType()),
+ Field.notNullable("update_rule", MinorType.UINT1.getType()),
+ Field.notNullable("delete_rule", MinorType.UINT1.getType())));
+ public static final Schema GET_IMPORTED_KEYS_SCHEMA = GET_IMPORTED_EXPORTED_AND_CROSS_REFERENCE_KEYS_SCHEMA;
+ public static final Schema GET_EXPORTED_KEYS_SCHEMA = GET_IMPORTED_EXPORTED_AND_CROSS_REFERENCE_KEYS_SCHEMA;
+ public static final Schema GET_CROSS_REFERENCE_SCHEMA = GET_IMPORTED_EXPORTED_AND_CROSS_REFERENCE_KEYS_SCHEMA;
+ private static final List GET_SQL_INFO_DENSE_UNION_SCHEMA_FIELDS = asList(
+ Field.nullable("string_value", VARCHAR.getType()),
+ Field.nullable("bool_value", BIT.getType()),
+ Field.nullable("bigint_value", BIGINT.getType()),
+ Field.nullable("int32_bitmask", INT.getType()),
+ new Field(
+ "string_list", FieldType.nullable(LIST.getType()),
+ singletonList(Field.nullable(ListVector.DATA_VECTOR_NAME, VARCHAR.getType()))),
+ new Field(
+ "int32_to_int32_list_map", FieldType.nullable(new ArrowType.Map(false)),
+ singletonList(new Field(DATA_VECTOR_NAME, new FieldType(false, STRUCT.getType(), null),
+ ImmutableList.of(
+ Field.notNullable(KEY_NAME, INT.getType()),
+ new Field(
+ VALUE_NAME, FieldType.nullable(LIST.getType()),
+ singletonList(Field.nullable(ListVector.DATA_VECTOR_NAME, INT.getType()))))))));
+ public static final Schema GET_SQL_INFO_SCHEMA =
+ new Schema(asList(
+ Field.notNullable("info_name", UINT4.getType()),
+ new Field("value",
+ FieldType.nullable(
+ new Union(UnionMode.Dense, range(0, GET_SQL_INFO_DENSE_UNION_SCHEMA_FIELDS.size()).toArray())),
+ GET_SQL_INFO_DENSE_UNION_SCHEMA_FIELDS)));
+ public static final Schema GET_PRIMARY_KEYS_SCHEMA =
+ new Schema(asList(
+ Field.nullable("catalog_name", VARCHAR.getType()),
+ Field.nullable("schema_name", VARCHAR.getType()),
+ Field.notNullable("table_name", VARCHAR.getType()),
+ Field.notNullable("column_name", VARCHAR.getType()),
+ Field.notNullable("key_sequence", INT.getType()),
+ Field.nullable("key_name", VARCHAR.getType())));
+
+ private Schemas() {
+ // Prevent instantiation.
+ }
+ }
+}
diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlUtils.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlUtils.java
new file mode 100644
index 0000000000000..25affa8f08aaa
--- /dev/null
+++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlUtils.java
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.List;
+
+import org.apache.arrow.flight.ActionType;
+import org.apache.arrow.flight.CallStatus;
+
+import com.google.common.collect.ImmutableList;
+import com.google.protobuf.Any;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.Message;
+
+/**
+ * Utilities to work with Flight SQL semantics.
+ */
+public final class FlightSqlUtils {
+ public static final ActionType FLIGHT_SQL_CREATE_PREPARED_STATEMENT = new ActionType("CreatePreparedStatement",
+ "Creates a reusable prepared statement resource on the server. \n" +
+ "Request Message: ActionCreatePreparedStatementRequest\n" +
+ "Response Message: ActionCreatePreparedStatementResult");
+
+ public static final ActionType FLIGHT_SQL_CLOSE_PREPARED_STATEMENT = new ActionType("ClosePreparedStatement",
+ "Closes a reusable prepared statement resource on the server. \n" +
+ "Request Message: ActionClosePreparedStatementRequest\n" +
+ "Response Message: N/A");
+
+ public static final List FLIGHT_SQL_ACTIONS = ImmutableList.of(
+ FLIGHT_SQL_CREATE_PREPARED_STATEMENT,
+ FLIGHT_SQL_CLOSE_PREPARED_STATEMENT
+ );
+
+ /**
+ * Helper to parse {@link com.google.protobuf.Any} objects to the specific protobuf object.
+ *
+ * @param source the raw bytes source value.
+ * @return the materialized protobuf object.
+ */
+ public static Any parseOrThrow(byte[] source) {
+ try {
+ return Any.parseFrom(source);
+ } catch (final InvalidProtocolBufferException e) {
+ throw CallStatus.INVALID_ARGUMENT
+ .withDescription("Received invalid message from remote.")
+ .withCause(e)
+ .toRuntimeException();
+ }
+ }
+
+ /**
+ * Helper to unpack {@link com.google.protobuf.Any} objects to the specific protobuf object.
+ *
+ * @param source the parsed Source value.
+ * @param as the class to unpack as.
+ * @param the class to unpack as.
+ * @return the materialized protobuf object.
+ */
+ public static T unpackOrThrow(Any source, Class as) {
+ try {
+ return source.unpack(as);
+ } catch (final InvalidProtocolBufferException e) {
+ throw CallStatus.INVALID_ARGUMENT
+ .withDescription("Provided message cannot be unpacked as desired type.")
+ .withCause(e)
+ .toRuntimeException();
+ }
+ }
+
+ /**
+ * Helper to parse and unpack {@link com.google.protobuf.Any} objects to the specific protobuf object.
+ *
+ * @param source the raw bytes source value.
+ * @param as the class to unpack as.
+ * @param the class to unpack as.
+ * @return the materialized protobuf object.
+ */
+ public static T unpackAndParseOrThrow(byte[] source, Class as) {
+ return unpackOrThrow(parseOrThrow(source), as);
+ }
+}
diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/SqlInfoBuilder.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/SqlInfoBuilder.java
new file mode 100644
index 0000000000000..3866cb89b1f21
--- /dev/null
+++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/SqlInfoBuilder.java
@@ -0,0 +1,1024 @@
+/*
+ * 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 java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.stream.IntStream.range;
+import static org.apache.arrow.flight.FlightProducer.ServerStreamListener;
+import static org.apache.arrow.flight.sql.util.SqlInfoOptionsUtils.createBitmaskFromEnums;
+
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Consumer;
+import java.util.function.ObjIntConsumer;
+
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlInfo;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlOuterJoinsSupportLevel;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlSupportedCaseSensitivity;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlSupportedElementActions;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlSupportedGroupBy;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlSupportedPositionedCommands;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlSupportedResultSetType;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlSupportedSubqueries;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlSupportedUnions;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlTransactionIsolationLevel;
+import org.apache.arrow.flight.sql.impl.FlightSql.SupportedAnsi92SqlGrammarLevel;
+import org.apache.arrow.flight.sql.impl.FlightSql.SupportedSqlGrammar;
+import org.apache.arrow.memory.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.complex.DenseUnionVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.MapVector;
+import org.apache.arrow.vector.complex.impl.UnionListWriter;
+import org.apache.arrow.vector.complex.impl.UnionMapWriter;
+import org.apache.arrow.vector.complex.writer.BaseWriter;
+import org.apache.arrow.vector.holders.NullableBigIntHolder;
+import org.apache.arrow.vector.holders.NullableBitHolder;
+import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.NullableVarCharHolder;
+
+import com.google.protobuf.ProtocolMessageEnum;
+
+/**
+ * Auxiliary class meant to facilitate the implementation of {@link FlightSqlProducer#getStreamSqlInfo}.
+ *
+ * Usage requires the user to add the required SqlInfo values using the {@code with*} methods
+ * like {@link SqlInfoBuilder#withFlightSqlServerName(String)}, and request it back
+ * through the {@link SqlInfoBuilder#send(List, ServerStreamListener)} method.
+ */
+@SuppressWarnings({"unused"})
+public class SqlInfoBuilder {
+ private final Map> providers = new HashMap<>();
+
+ /**
+ * Gets a {@link NullableVarCharHolder} from the provided {@code string} using the provided {@code buf}.
+ *
+ * @param string the {@link StandardCharsets#UTF_8}-encoded text input to store onto the holder.
+ * @param buf the {@link ArrowBuf} from which to create the new holder.
+ * @return a new {@link NullableVarCharHolder} with the provided input data {@code string}.
+ */
+ public static NullableVarCharHolder getHolderForUtf8(final String string, final ArrowBuf buf) {
+ final byte[] bytes = string.getBytes(UTF_8);
+ buf.setBytes(0, bytes);
+ final NullableVarCharHolder holder = new NullableVarCharHolder();
+ holder.buffer = buf;
+ holder.end = bytes.length;
+ holder.isSet = 1;
+ return holder;
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#FLIGHT_SQL_SERVER_NAME} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#FLIGHT_SQL_SERVER_NAME} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withFlightSqlServerName(final String value) {
+ return withStringProvider(SqlInfo.FLIGHT_SQL_SERVER_NAME_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#FLIGHT_SQL_SERVER_VERSION} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#FLIGHT_SQL_SERVER_VERSION} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withFlightSqlServerVersion(final String value) {
+ return withStringProvider(SqlInfo.FLIGHT_SQL_SERVER_VERSION_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#FLIGHT_SQL_SERVER_ARROW_VERSION} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#FLIGHT_SQL_SERVER_ARROW_VERSION} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withFlightSqlServerArrowVersion(final String value) {
+ return withStringProvider(SqlInfo.FLIGHT_SQL_SERVER_ARROW_VERSION_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_IDENTIFIER_QUOTE_CHAR} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_IDENTIFIER_QUOTE_CHAR} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlIdentifierQuoteChar(final String value) {
+ return withStringProvider(SqlInfo.SQL_IDENTIFIER_QUOTE_CHAR_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SEARCH_STRING_ESCAPE} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_SEARCH_STRING_ESCAPE} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSearchStringEscape(final String value) {
+ return withStringProvider(SqlInfo.SQL_SEARCH_STRING_ESCAPE_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_EXTRA_NAME_CHARACTERS} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_EXTRA_NAME_CHARACTERS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlExtraNameCharacters(final String value) {
+ return withStringProvider(SqlInfo.SQL_EXTRA_NAME_CHARACTERS_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SCHEMA_TERM} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_SCHEMA_TERM} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSchemaTerm(final String value) {
+ return withStringProvider(SqlInfo.SQL_SCHEMA_TERM_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_CATALOG_TERM} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_CATALOG_TERM} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlCatalogTerm(final String value) {
+ return withStringProvider(SqlInfo.SQL_CATALOG_TERM_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_PROCEDURE_TERM} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_PROCEDURE_TERM} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlProcedureTerm(final String value) {
+ return withStringProvider(SqlInfo.SQL_PROCEDURE_TERM_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_DDL_CATALOG} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_DDL_CATALOG} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlDdlCatalog(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_DDL_CATALOG_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_DDL_SCHEMA} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_DDL_SCHEMA} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlDdlSchema(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_DDL_SCHEMA_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_DDL_TABLE} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_DDL_TABLE} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlDdlTable(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_DDL_TABLE_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#FLIGHT_SQL_SERVER_READ_ONLY} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#FLIGHT_SQL_SERVER_READ_ONLY} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withFlightSqlServerReadOnly(final boolean value) {
+ return withBooleanProvider(SqlInfo.FLIGHT_SQL_SERVER_READ_ONLY_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTS_COLUMN_ALIASING} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_SUPPORTS_COLUMN_ALIASING} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportsColumnAliasing(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_SUPPORTS_COLUMN_ALIASING_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_NULL_PLUS_NULL_IS_NULL} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_NULL_PLUS_NULL_IS_NULL} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlNullPlusNullIsNull(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_NULL_PLUS_NULL_IS_NULL_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTS_TABLE_CORRELATION_NAMES} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_SUPPORTS_TABLE_CORRELATION_NAMES} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportsTableCorrelationNames(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_SUPPORTS_TABLE_CORRELATION_NAMES_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTS_DIFFERENT_TABLE_CORRELATION_NAMES} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_SUPPORTS_DIFFERENT_TABLE_CORRELATION_NAMES} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportsDifferentTableCorrelationNames(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_SUPPORTS_DIFFERENT_TABLE_CORRELATION_NAMES_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTS_EXPRESSIONS_IN_ORDER_BY} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_SUPPORTS_EXPRESSIONS_IN_ORDER_BY} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportsExpressionsInOrderBy(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_SUPPORTS_EXPRESSIONS_IN_ORDER_BY_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTS_ORDER_BY_UNRELATED} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_SUPPORTS_ORDER_BY_UNRELATED} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportsOrderByUnrelated(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_SUPPORTS_ORDER_BY_UNRELATED_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTS_LIKE_ESCAPE_CLAUSE} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_SUPPORTS_LIKE_ESCAPE_CLAUSE} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportsLikeEscapeClause(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_SUPPORTS_LIKE_ESCAPE_CLAUSE_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTS_NON_NULLABLE_COLUMNS} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_SUPPORTS_NON_NULLABLE_COLUMNS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportsNonNullableColumns(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_SUPPORTS_NON_NULLABLE_COLUMNS_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTS_INTEGRITY_ENHANCEMENT_FACILITY} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_SUPPORTS_INTEGRITY_ENHANCEMENT_FACILITY} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportsIntegrityEnhancementFacility(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_SUPPORTS_INTEGRITY_ENHANCEMENT_FACILITY_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_CATALOG_AT_START} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_CATALOG_AT_START} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlCatalogAtStart(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_CATALOG_AT_START_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SELECT_FOR_UPDATE_SUPPORTED} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_SELECT_FOR_UPDATE_SUPPORTED} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSelectForUpdateSupported(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_SELECT_FOR_UPDATE_SUPPORTED_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_STORED_PROCEDURES_SUPPORTED} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_STORED_PROCEDURES_SUPPORTED} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlStoredProceduresSupported(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_STORED_PROCEDURES_SUPPORTED_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_CORRELATED_SUBQUERIES_SUPPORTED} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_CORRELATED_SUBQUERIES_SUPPORTED} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlCorrelatedSubqueriesSupported(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_CORRELATED_SUBQUERIES_SUPPORTED_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_ROW_SIZE_INCLUDES_BLOBS} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_ROW_SIZE_INCLUDES_BLOBS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxRowSizeIncludesBlobs(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_MAX_ROW_SIZE_INCLUDES_BLOBS_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_TRANSACTIONS_SUPPORTED} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_TRANSACTIONS_SUPPORTED} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlTransactionsSupported(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_TRANSACTIONS_SUPPORTED_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_DATA_DEFINITION_CAUSES_TRANSACTION_COMMIT} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_DATA_DEFINITION_CAUSES_TRANSACTION_COMMIT} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlDataDefinitionCausesTransactionCommit(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_DATA_DEFINITION_CAUSES_TRANSACTION_COMMIT_VALUE,
+ value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_DATA_DEFINITIONS_IN_TRANSACTIONS_IGNORED} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_DATA_DEFINITIONS_IN_TRANSACTIONS_IGNORED} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlDataDefinitionsInTransactionsIgnored(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_DATA_DEFINITIONS_IN_TRANSACTIONS_IGNORED_VALUE,
+ value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_BATCH_UPDATES_SUPPORTED} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_BATCH_UPDATES_SUPPORTED} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlBatchUpdatesSupported(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_BATCH_UPDATES_SUPPORTED_VALUE, value);
+ }
+
+ /**
+ * Sets a value for { @link SqlInfo#SQL_SAVEPOINTS_SUPPORTED} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_SAVEPOINTS_SUPPORTED} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSavepointsSupported(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_SAVEPOINTS_SUPPORTED_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_NAMED_PARAMETERS_SUPPORTED} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_NAMED_PARAMETERS_SUPPORTED} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlNamedParametersSupported(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_NAMED_PARAMETERS_SUPPORTED_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_LOCATORS_UPDATE_COPY} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_LOCATORS_UPDATE_COPY} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlLocatorsUpdateCopy(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_LOCATORS_UPDATE_COPY_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_STORED_FUNCTIONS_USING_CALL_SYNTAX_SUPPORTED} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_STORED_FUNCTIONS_USING_CALL_SYNTAX_SUPPORTED} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlStoredFunctionsUsingCallSyntaxSupported(final boolean value) {
+ return withBooleanProvider(SqlInfo.SQL_STORED_FUNCTIONS_USING_CALL_SYNTAX_SUPPORTED_VALUE,
+ value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_IDENTIFIER_CASE} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_IDENTIFIER_CASE} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlIdentifierCase(final SqlSupportedCaseSensitivity value) {
+ return withBitIntProvider(SqlInfo.SQL_IDENTIFIER_CASE_VALUE, value.getNumber());
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_QUOTED_IDENTIFIER_CASE} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_QUOTED_IDENTIFIER_CASE} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlQuotedIdentifierCase(final SqlSupportedCaseSensitivity value) {
+ return withBitIntProvider(SqlInfo.SQL_QUOTED_IDENTIFIER_CASE_VALUE, value.getNumber());
+ }
+
+ /**
+ * Sets a value SqlInf @link SqlInfo#SQL_MAX_BINARY_LITERAL_LENGTH} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_BINARY_LITERAL_LENGTH} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxBinaryLiteralLength(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_BINARY_LITERAL_LENGTH_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_CHAR_LITERAL_LENGTH} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_CHAR_LITERAL_LENGTH} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxCharLiteralLength(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_CHAR_LITERAL_LENGTH_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_COLUMN_NAME_LENGTH} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_COLUMN_NAME_LENGTH} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxColumnNameLength(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_COLUMN_NAME_LENGTH_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_COLUMNS_IN_GROUP_BY} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_COLUMNS_IN_GROUP_BY} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxColumnsInGroupBy(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_COLUMNS_IN_GROUP_BY_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_COLUMNS_IN_INDEX} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_COLUMNS_IN_INDEX} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxColumnsInIndex(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_COLUMNS_IN_INDEX_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_COLUMNS_IN_ORDER_BY} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_COLUMNS_IN_ORDER_BY} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxColumnsInOrderBy(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_COLUMNS_IN_ORDER_BY_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_COLUMNS_IN_SELECT} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_COLUMNS_IN_SELECT} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxColumnsInSelect(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_COLUMNS_IN_SELECT_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_CONNECTIONS} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_CONNECTIONS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxConnections(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_CONNECTIONS_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_CURSOR_NAME_LENGTH} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_CURSOR_NAME_LENGTH} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxCursorNameLength(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_CURSOR_NAME_LENGTH_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_INDEX_LENGTH} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_INDEX_LENGTH} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxIndexLength(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_INDEX_LENGTH_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_DB_SCHEMA_NAME_LENGTH} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_DB_SCHEMA_NAME_LENGTH} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlDbSchemaNameLength(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_DB_SCHEMA_NAME_LENGTH_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_PROCEDURE_NAME_LENGTH} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_PROCEDURE_NAME_LENGTH} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxProcedureNameLength(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_PROCEDURE_NAME_LENGTH_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_CATALOG_NAME_LENGTH} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_CATALOG_NAME_LENGTH} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxCatalogNameLength(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_CATALOG_NAME_LENGTH_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_ROW_SIZE} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_ROW_SIZE} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxRowSize(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_ROW_SIZE_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_STATEMENT_LENGTH} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_STATEMENT_LENGTH} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxStatementLength(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_STATEMENT_LENGTH_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_STATEMENTS} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_STATEMENTS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxStatements(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_STATEMENTS_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_TABLE_NAME_LENGTH} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_TABLE_NAME_LENGTH} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxTableNameLength(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_TABLE_NAME_LENGTH_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_TABLES_IN_SELECT} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_TABLES_IN_SELECT} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxTablesInSelect(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_TABLES_IN_SELECT_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_MAX_USERNAME_LENGTH} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_MAX_USERNAME_LENGTH} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlMaxUsernameLength(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_MAX_USERNAME_LENGTH_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_DEFAULT_TRANSACTION_ISOLATION} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_DEFAULT_TRANSACTION_ISOLATION} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlDefaultTransactionIsolation(final long value) {
+ return withBitIntProvider(SqlInfo.SQL_DEFAULT_TRANSACTION_ISOLATION_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTED_GROUP_BY} in the builder.
+ *
+ * @param values the value for {@link SqlInfo#SQL_SUPPORTED_GROUP_BY} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportedGroupBy(final SqlSupportedGroupBy... values) {
+ return withEnumProvider(SqlInfo.SQL_SUPPORTED_GROUP_BY_VALUE, values);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTED_GRAMMAR} in the builder.
+ *
+ * @param values the value for {@link SqlInfo#SQL_SUPPORTED_GRAMMAR} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportedGrammar(final SupportedSqlGrammar... values) {
+ return withEnumProvider(SqlInfo.SQL_SUPPORTED_GRAMMAR_VALUE, values);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_ANSI92_SUPPORTED_LEVEL} in the builder.
+ *
+ * @param values the value for {@link SqlInfo#SQL_ANSI92_SUPPORTED_LEVEL} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlAnsi92SupportedLevel(final SupportedAnsi92SqlGrammarLevel... values) {
+ return withEnumProvider(SqlInfo.SQL_ANSI92_SUPPORTED_LEVEL_VALUE, values);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SCHEMAS_SUPPORTED_ACTIONS} in the builder.
+ *
+ * @param values the value for {@link SqlInfo#SQL_SCHEMAS_SUPPORTED_ACTIONS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSchemasSupportedActions(final SqlSupportedElementActions... values) {
+ return withEnumProvider(SqlInfo.SQL_SCHEMAS_SUPPORTED_ACTIONS_VALUE, values);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_CATALOGS_SUPPORTED_ACTIONS} in the builder.
+ *
+ * @param values the value for {@link SqlInfo#SQL_CATALOGS_SUPPORTED_ACTIONS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlCatalogsSupportedActions(final SqlSupportedElementActions... values) {
+ return withEnumProvider(SqlInfo.SQL_CATALOGS_SUPPORTED_ACTIONS_VALUE, values);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTED_POSITIONED_COMMANDS} in the builder.
+ *
+ * @param values the value for {@link SqlInfo#SQL_SUPPORTED_POSITIONED_COMMANDS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportedPositionedCommands(final SqlSupportedPositionedCommands... values) {
+ return withEnumProvider(SqlInfo.SQL_SUPPORTED_POSITIONED_COMMANDS_VALUE, values);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTED_SUBQUERIES} in the builder.
+ *
+ * @param values the value for {@link SqlInfo#SQL_SUPPORTED_SUBQUERIES} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSubQueriesSupported(final SqlSupportedSubqueries... values) {
+ return withEnumProvider(SqlInfo.SQL_SUPPORTED_SUBQUERIES_VALUE, values);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTED_UNIONS} in the builder.
+ *
+ * @param values the values for {@link SqlInfo#SQL_SUPPORTED_UNIONS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportedUnions(final SqlSupportedUnions... values) {
+ return withEnumProvider(SqlInfo.SQL_SUPPORTED_UNIONS_VALUE, values);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_OUTER_JOINS_SUPPORT_LEVEL} in the builder.
+ *
+ * @param value the value for {@link SqlInfo#SQL_OUTER_JOINS_SUPPORT_LEVEL} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlOuterJoinSupportLevel(final SqlOuterJoinsSupportLevel... value) {
+ return withEnumProvider(SqlInfo.SQL_OUTER_JOINS_SUPPORT_LEVEL_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTED_TRANSACTIONS_ISOLATION_LEVELS} in the builder.
+ *
+ * @param values the values for {@link SqlInfo#SQL_SUPPORTED_TRANSACTIONS_ISOLATION_LEVELS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportedTransactionsIsolationLevels(final SqlTransactionIsolationLevel... values) {
+ return withEnumProvider(SqlInfo.SQL_SUPPORTED_TRANSACTIONS_ISOLATION_LEVELS_VALUE, values);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTED_RESULT_SET_TYPES} in the builder.
+ *
+ * @param values the values for {@link SqlInfo#SQL_SUPPORTED_RESULT_SET_TYPES} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportedResultSetTypes(final SqlSupportedResultSetType... values) {
+ return withEnumProvider(SqlInfo.SQL_SUPPORTED_RESULT_SET_TYPES_VALUE, values
+ );
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_KEYWORDS} in the builder.
+ *
+ * @param value the values for {@link SqlInfo#SQL_KEYWORDS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlKeywords(final String[] value) {
+ return withStringArrayProvider(SqlInfo.SQL_KEYWORDS_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_NUMERIC_FUNCTIONS} in the builder.
+ *
+ * @param value the values for {@link SqlInfo#SQL_NUMERIC_FUNCTIONS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlNumericFunctions(final String[] value) {
+ return withStringArrayProvider(SqlInfo.SQL_NUMERIC_FUNCTIONS_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_STRING_FUNCTIONS} in the builder.
+ *
+ * @param value the values for {@link SqlInfo#SQL_STRING_FUNCTIONS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlStringFunctions(final String[] value) {
+ return withStringArrayProvider(SqlInfo.SQL_STRING_FUNCTIONS_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SYSTEM_FUNCTIONS} in the builder.
+ *
+ * @param value the values for {@link SqlInfo#SQL_SYSTEM_FUNCTIONS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSystemFunctions(final String[] value) {
+ return withStringArrayProvider(SqlInfo.SQL_SYSTEM_FUNCTIONS_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_DATETIME_FUNCTIONS} in the builder.
+ *
+ * @param value the values for {@link SqlInfo#SQL_DATETIME_FUNCTIONS} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlDatetimeFunctions(final String[] value) {
+ return withStringArrayProvider(SqlInfo.SQL_DATETIME_FUNCTIONS_VALUE, value);
+ }
+
+ /**
+ * Sets a value for {@link SqlInfo#SQL_SUPPORTS_CONVERT} in the builder.
+ *
+ * @param value the values for {@link SqlInfo#SQL_SUPPORTS_CONVERT} to be set.
+ * @return the SqlInfoBuilder itself.
+ */
+ public SqlInfoBuilder withSqlSupportsConvert(final Map> value) {
+ return withIntToIntListMapProvider(SqlInfo.SQL_SUPPORTS_CONVERT_VALUE, value);
+ }
+
+ private void addProvider(final int sqlInfo, final ObjIntConsumer provider) {
+ providers.put(sqlInfo, provider);
+ }
+
+ private SqlInfoBuilder withEnumProvider(final int sqlInfo, final ProtocolMessageEnum[] values) {
+ return withIntProvider(sqlInfo, (int) createBitmaskFromEnums(values));
+ }
+
+ private SqlInfoBuilder withIntProvider(final int sqlInfo, final int value) {
+ addProvider(sqlInfo, (root, index) -> setDataForIntField(root, index, sqlInfo, value));
+ return this;
+ }
+
+ private SqlInfoBuilder withBitIntProvider(final int sqlInfo, final long value) {
+ addProvider(sqlInfo, (root, index) -> setDataForBigIntField(root, index, sqlInfo, value));
+ return this;
+ }
+
+ private SqlInfoBuilder withBooleanProvider(final int sqlInfo,
+ final boolean value) {
+ addProvider(sqlInfo, (root, index) -> setDataForBooleanField(root, index, sqlInfo, value));
+ return this;
+ }
+
+ private SqlInfoBuilder withStringProvider(final int sqlInfo, final String value) {
+ addProvider(sqlInfo, (root, index) -> setDataForUtf8Field(root, index, sqlInfo, value));
+ return this;
+ }
+
+ private SqlInfoBuilder withStringArrayProvider(final int sqlInfo,
+ final String[] value) {
+ addProvider(sqlInfo, (root, index) -> setDataVarCharListField(root, index, sqlInfo, value));
+ return this;
+ }
+
+ private SqlInfoBuilder withIntToIntListMapProvider(final int sqlInfo,
+ final Map> value) {
+ addProvider(sqlInfo, (root, index) -> setIntToIntListMapField(root, index, sqlInfo, value));
+ return this;
+ }
+
+ /**
+ * Send the requested information to given ServerStreamListener.
+ *
+ * @param infos List of SqlInfo to be sent.
+ * @param listener ServerStreamListener to send data to.
+ */
+ public void send(List infos, final ServerStreamListener listener) {
+ if (infos == null || infos.isEmpty()) {
+ infos = new ArrayList<>(providers.keySet());
+ }
+ try (final BufferAllocator allocator = new RootAllocator();
+ final VectorSchemaRoot root = VectorSchemaRoot.create(
+ FlightSqlProducer.Schemas.GET_SQL_INFO_SCHEMA,
+ allocator)) {
+ final int rows = infos.size();
+ for (int i = 0; i < rows; i++) {
+ providers.get(infos.get(i)).accept(root, i);
+ }
+ root.setRowCount(rows);
+ listener.start(root);
+ listener.putNext();
+ } catch (final Throwable throwable) {
+ listener.error(throwable);
+ } finally {
+ listener.completed();
+ }
+ }
+
+ private void setInfoName(final VectorSchemaRoot root, final int index, final int info) {
+ final UInt4Vector infoName = (UInt4Vector) root.getVector("info_name");
+ infoName.setSafe(index, info);
+ }
+
+ private void setValues(final VectorSchemaRoot root, final int index, final byte typeId,
+ final Consumer dataSetter) {
+ final DenseUnionVector values = (DenseUnionVector) root.getVector("value");
+ values.setTypeId(index, typeId);
+ dataSetter.accept(values);
+ }
+
+ /**
+ * Executes the given action on an ad-hoc, newly created instance of {@link ArrowBuf}.
+ *
+ * @param executor the action to take.
+ */
+ private void onCreateArrowBuf(final Consumer executor) {
+ try (final BufferAllocator allocator = new RootAllocator();
+ final ArrowBuf buf = allocator.buffer(1024)) {
+ executor.accept(buf);
+ }
+ }
+
+ private void setDataForUtf8Field(final VectorSchemaRoot root, final int index,
+ final int sqlInfo, final String value) {
+ setInfoName(root, index, sqlInfo);
+ onCreateArrowBuf(buf -> {
+ final Consumer producer =
+ values -> values.setSafe(index, getHolderForUtf8(value, buf));
+ setValues(root, index, (byte) 0, producer);
+ });
+ }
+
+ private void setDataForIntField(final VectorSchemaRoot root, final int index,
+ final int sqlInfo, final int value) {
+ setInfoName(root, index, sqlInfo);
+ final NullableIntHolder dataHolder = new NullableIntHolder();
+ dataHolder.isSet = 1;
+ dataHolder.value = value;
+ setValues(root, index, (byte) 3, values -> values.setSafe(index, dataHolder));
+ }
+
+ private void setDataForBigIntField(final VectorSchemaRoot root, final int index,
+ final int sqlInfo, final long value) {
+ setInfoName(root, index, sqlInfo);
+ final NullableBigIntHolder dataHolder = new NullableBigIntHolder();
+ dataHolder.isSet = 1;
+ dataHolder.value = value;
+ setValues(root, index, (byte) 2, values -> values.setSafe(index, dataHolder));
+ }
+
+ private void setDataForBooleanField(final VectorSchemaRoot root, final int index,
+ final int sqlInfo, final boolean value) {
+ setInfoName(root, index, sqlInfo);
+ final NullableBitHolder dataHolder = new NullableBitHolder();
+ dataHolder.isSet = 1;
+ dataHolder.value = value ? 1 : 0;
+ setValues(root, index, (byte) 1, values -> values.setSafe(index, dataHolder));
+ }
+
+ private void setDataVarCharListField(final VectorSchemaRoot root, final int index,
+ final int sqlInfo,
+ final String[] values) {
+ final DenseUnionVector denseUnion = (DenseUnionVector) root.getVector("value");
+ final ListVector listVector = denseUnion.getList((byte) 4);
+ final int listIndex = listVector.getValueCount();
+ final int denseUnionValueCount = index + 1;
+ final int listVectorValueCount = listIndex + 1;
+ denseUnion.setValueCount(denseUnionValueCount);
+ listVector.setValueCount(listVectorValueCount);
+
+ final UnionListWriter writer = listVector.getWriter();
+ writer.setPosition(listIndex);
+ writer.startList();
+ final int length = values.length;
+ range(0, length)
+ .forEach(i -> onCreateArrowBuf(buf -> {
+ final byte[] bytes = values[i].getBytes(UTF_8);
+ buf.setBytes(0, bytes);
+ writer.writeVarChar(0, bytes.length, buf);
+ }));
+ writer.endList();
+ writer.setValueCount(listVectorValueCount);
+
+ denseUnion.setTypeId(index, (byte) 4);
+ denseUnion.getOffsetBuffer().setInt(index * 4L, listIndex);
+ setInfoName(root, index, sqlInfo);
+ }
+
+ private void setIntToIntListMapField(final VectorSchemaRoot root, final int index,
+ final int sqlInfo,
+ final Map> values) {
+ final DenseUnionVector denseUnion = (DenseUnionVector) root.getVector("value");
+ final MapVector mapVector = denseUnion.getMap((byte) 5);
+ final int mapIndex = mapVector.getValueCount();
+ denseUnion.setValueCount(index + 1);
+ mapVector.setValueCount(mapIndex + 1);
+
+ final UnionMapWriter mapWriter = mapVector.getWriter();
+ mapWriter.setPosition(mapIndex);
+ mapWriter.startMap();
+ values.forEach((key, value) -> {
+ mapWriter.startEntry();
+ mapWriter.key().integer().writeInt(key);
+ final BaseWriter.ListWriter listWriter = mapWriter.value().list();
+ listWriter.startList();
+ for (final int v : value) {
+ listWriter.integer().writeInt(v);
+ }
+ listWriter.endList();
+ mapWriter.endEntry();
+ });
+ mapWriter.endMap();
+ mapWriter.setValueCount(mapIndex + 1);
+
+ denseUnion.setTypeId(index, (byte) 5);
+ denseUnion.getOffsetBuffer().setInt(index * 4L, mapIndex);
+ setInfoName(root, index, sqlInfo);
+ }
+}
diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/example/FlightSqlClientDemoApp.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/example/FlightSqlClientDemoApp.java
new file mode 100644
index 0000000000000..f3774a8a50012
--- /dev/null
+++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/example/FlightSqlClientDemoApp.java
@@ -0,0 +1,244 @@
+/*
+ * 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.util.ArrayList;
+import java.util.List;
+
+import org.apache.arrow.flight.CallOption;
+import org.apache.arrow.flight.FlightClient;
+import org.apache.arrow.flight.FlightInfo;
+import org.apache.arrow.flight.FlightStream;
+import org.apache.arrow.flight.Location;
+import org.apache.arrow.flight.sql.FlightSqlClient;
+import org.apache.arrow.flight.sql.util.TableRef;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+
+/**
+ * Flight SQL Client Demo CLI Application.
+ */
+public class FlightSqlClientDemoApp implements AutoCloseable {
+ public final List callOptions = new ArrayList<>();
+ public final BufferAllocator allocator;
+ public FlightSqlClient flightSqlClient;
+
+ public FlightSqlClientDemoApp(final BufferAllocator bufferAllocator) {
+ allocator = bufferAllocator;
+ }
+
+ public static void main(final String[] args) throws Exception {
+ final Options options = new Options();
+
+ options.addRequiredOption("host", "host", true, "Host to connect to");
+ options.addRequiredOption("port", "port", true, "Port to connect to");
+ options.addRequiredOption("command", "command", true, "Method to run");
+
+ options.addOption("query", "query", false, "Query");
+ options.addOption("catalog", "catalog", false, "Catalog");
+ options.addOption("schema", "schema", false, "Schema");
+ options.addOption("table", "table", false, "Table");
+
+ CommandLineParser parser = new DefaultParser();
+ HelpFormatter formatter = new HelpFormatter();
+ CommandLine cmd;
+
+ try {
+ cmd = parser.parse(options, args);
+ try (final FlightSqlClientDemoApp thisApp = new FlightSqlClientDemoApp(new RootAllocator(Integer.MAX_VALUE))) {
+ thisApp.executeApp(cmd);
+ }
+
+ } catch (final ParseException e) {
+ System.out.println(e.getMessage());
+ formatter.printHelp("FlightSqlClientDemoApp -host localhost -port 32010 ...", options);
+ throw e;
+ }
+ }
+
+ /**
+ * Gets the current {@link CallOption} as an array; usually used as an
+ * argument in {@link FlightSqlClient} methods.
+ *
+ * @return current {@link CallOption} array.
+ */
+ public CallOption[] getCallOptions() {
+ return callOptions.toArray(new CallOption[0]);
+ }
+
+ /**
+ * Calls {@link FlightSqlClientDemoApp#createFlightSqlClient(String, int)}
+ * in order to create a {@link FlightSqlClient} to be used in future calls,
+ * and then calls {@link FlightSqlClientDemoApp#executeCommand(CommandLine)}
+ * to execute the command parsed at execution.
+ *
+ * @param cmd parsed {@link CommandLine}; often the result of {@link DefaultParser#parse(Options, String[])}.
+ */
+ public void executeApp(final CommandLine cmd) throws Exception {
+ final String host = cmd.getOptionValue("host").trim();
+ final int port = Integer.parseInt(cmd.getOptionValue("port").trim());
+
+ createFlightSqlClient(host, port);
+ executeCommand(cmd);
+ }
+
+ /**
+ * Parses the "{@code command}" CLI argument and redirects to the appropriate method.
+ *
+ * @param cmd parsed {@link CommandLine}; often the result of
+ * {@link DefaultParser#parse(Options, String[])}.
+ */
+ public void executeCommand(CommandLine cmd) throws Exception {
+ switch (cmd.getOptionValue("command").trim()) {
+ case "Execute":
+ exampleExecute(
+ cmd.getOptionValue("query")
+ );
+ break;
+ case "ExecuteUpdate":
+ exampleExecuteUpdate(
+ cmd.getOptionValue("query")
+ );
+ break;
+ case "GetCatalogs":
+ exampleGetCatalogs();
+ break;
+ case "GetSchemas":
+ exampleGetSchemas(
+ cmd.getOptionValue("catalog"),
+ cmd.getOptionValue("schema")
+ );
+ break;
+ case "GetTableTypes":
+ exampleGetTableTypes();
+ break;
+ case "GetTables":
+ exampleGetTables(
+ cmd.getOptionValue("catalog"),
+ cmd.getOptionValue("schema"),
+ cmd.getOptionValue("table")
+ );
+ break;
+ case "GetExportedKeys":
+ exampleGetExportedKeys(
+ cmd.getOptionValue("catalog"),
+ cmd.getOptionValue("schema"),
+ cmd.getOptionValue("table")
+ );
+ break;
+ case "GetImportedKeys":
+ exampleGetImportedKeys(
+ cmd.getOptionValue("catalog"),
+ cmd.getOptionValue("schema"),
+ cmd.getOptionValue("table")
+ );
+ break;
+ case "GetPrimaryKeys":
+ exampleGetPrimaryKeys(
+ cmd.getOptionValue("catalog"),
+ cmd.getOptionValue("schema"),
+ cmd.getOptionValue("table")
+ );
+ break;
+ default:
+ System.out.println("Command used is not valid! Please use one of: \n" +
+ "[\"ExecuteUpdate\",\n" +
+ "\"Execute\",\n" +
+ "\"GetCatalogs\",\n" +
+ "\"GetSchemas\",\n" +
+ "\"GetTableTypes\",\n" +
+ "\"GetTables\",\n" +
+ "\"GetExportedKeys\",\n" +
+ "\"GetImportedKeys\",\n" +
+ "\"GetPrimaryKeys\"]");
+ }
+ }
+
+ /**
+ * Creates a {@link FlightSqlClient} to be used with the example methods.
+ *
+ * @param host client's hostname.
+ * @param port client's port.
+ */
+ public void createFlightSqlClient(final String host, final int port) {
+ final Location clientLocation = Location.forGrpcInsecure(host, port);
+ flightSqlClient = new FlightSqlClient(FlightClient.builder(allocator, clientLocation).build());
+ }
+
+ private void exampleExecute(final String query) throws Exception {
+ printFlightInfoResults(flightSqlClient.execute(query, getCallOptions()));
+ }
+
+ private void exampleExecuteUpdate(final String query) {
+ System.out.println("Updated: " + flightSqlClient.executeUpdate(query, getCallOptions()) + "rows.");
+ }
+
+ private void exampleGetCatalogs() throws Exception {
+ printFlightInfoResults(flightSqlClient.getCatalogs(getCallOptions()));
+ }
+
+ private void exampleGetSchemas(final String catalog, final String schema) throws Exception {
+ printFlightInfoResults(flightSqlClient.getSchemas(catalog, schema, getCallOptions()));
+ }
+
+ private void exampleGetTableTypes() throws Exception {
+ printFlightInfoResults(flightSqlClient.getTableTypes(getCallOptions()));
+ }
+
+ private void exampleGetTables(final String catalog, final String schema, final String table) throws Exception {
+ // For now, this won't filter by table types.
+ printFlightInfoResults(flightSqlClient.getTables(
+ catalog, schema, table, null, false, getCallOptions()));
+ }
+
+ private void exampleGetExportedKeys(final String catalog, final String schema, final String table) throws Exception {
+ printFlightInfoResults(flightSqlClient.getExportedKeys(TableRef.of(catalog, schema, table), getCallOptions()));
+ }
+
+ private void exampleGetImportedKeys(final String catalog, final String schema, final String table) throws Exception {
+ printFlightInfoResults(flightSqlClient.getImportedKeys(TableRef.of(catalog, schema, table), getCallOptions()));
+ }
+
+ private void exampleGetPrimaryKeys(final String catalog, final String schema, final String table) throws Exception {
+ printFlightInfoResults(flightSqlClient.getPrimaryKeys(TableRef.of(catalog, schema, table), getCallOptions()));
+ }
+
+ private void printFlightInfoResults(final FlightInfo flightInfo) throws Exception {
+ final FlightStream stream =
+ flightSqlClient.getStream(flightInfo.getEndpoints().get(0).getTicket(), getCallOptions());
+ while (stream.next()) {
+ try (final VectorSchemaRoot root = stream.getRoot()) {
+ System.out.println(root.contentToTSVString());
+ }
+ }
+ stream.close();
+ }
+
+ @Override
+ public void close() throws Exception {
+ flightSqlClient.close();
+ allocator.close();
+ }
+}
diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/util/SqlInfoOptionsUtils.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/util/SqlInfoOptionsUtils.java
new file mode 100644
index 0000000000000..c43c48eb8e0dd
--- /dev/null
+++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/util/SqlInfoOptionsUtils.java
@@ -0,0 +1,71 @@
+/*
+ * 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.util;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.apache.arrow.flight.sql.FlightSqlClient;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlInfo;
+
+import com.google.protobuf.ProtocolMessageEnum;
+
+/**
+ * Utility class for {@link SqlInfo} and {@link FlightSqlClient#getSqlInfo} option parsing.
+ */
+public final class SqlInfoOptionsUtils {
+ private SqlInfoOptionsUtils() {
+ // Prevent instantiation.
+ }
+
+ /**
+ * Returns whether the provided {@code bitmask} points to the provided {@link ProtocolMessageEnum} by comparing
+ * {@link ProtocolMessageEnum#getNumber} with the respective bit index of the {@code bitmask}.
+ *
+ * @param enumInstance the protobuf message enum to use.
+ * @param bitmask the bitmask response from {@link FlightSqlClient#getSqlInfo}.
+ * @return whether the provided {@code bitmask} points to the specified {@code enumInstance}.
+ */
+ public static boolean doesBitmaskTranslateToEnum(final ProtocolMessageEnum enumInstance, final long bitmask) {
+ return ((bitmask >> enumInstance.getNumber()) & 1) == 1;
+ }
+
+ /**
+ * Creates a bitmask that translates to the specified {@code enums}.
+ *
+ * @param enums the {@link ProtocolMessageEnum} instances to represent as bitmask.
+ * @return the bitmask.
+ */
+ public static long createBitmaskFromEnums(final ProtocolMessageEnum... enums) {
+ return createBitmaskFromEnums(Arrays.asList(enums));
+ }
+
+ /**
+ * Creates a bitmask that translates to the specified {@code enums}.
+ *
+ * @param enums the {@link ProtocolMessageEnum} instances to represent as bitmask.
+ * @return the bitmask.
+ */
+ public static long createBitmaskFromEnums(final Collection enums) {
+ return enums.stream()
+ .mapToInt(ProtocolMessageEnum::getNumber)
+ .map(bitIndexToSet -> 1 << bitIndexToSet)
+ .reduce((firstBitmask, secondBitmask) -> firstBitmask | secondBitmask)
+ .orElse(0);
+ }
+}
diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/util/TableRef.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/util/TableRef.java
new file mode 100644
index 0000000000000..315f17ee911cf
--- /dev/null
+++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/util/TableRef.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight.sql.util;
+
+/**
+ * A helper class to reference a table to be passed to the flight
+ * sql client.
+ */
+public class TableRef {
+ private final String catalog;
+ private final String dbSchema;
+ private final String table;
+
+ /**
+ * The complete constructor for the TableRef class.
+ * @param catalog the catalog from a table.
+ * @param dbSchema the database schema from a table.
+ * @param table the table name from a table.
+ */
+ public TableRef(String catalog, String dbSchema, String table) {
+ this.catalog = catalog;
+ this.dbSchema = dbSchema;
+ this.table = table;
+ }
+
+ /**
+ * A static initializer of the TableRef with all the arguments.
+ * @param catalog the catalog from a table.
+ * @param dbSchema the database schema from a table.
+ * @param table the table name from a table.
+ * @return A TableRef object.
+ */
+ public static TableRef of(String catalog, String dbSchema, String table) {
+ return new TableRef(catalog, dbSchema, table);
+ }
+
+ /**
+ * Retrieve the catalog from the object.
+ * @return the catalog.
+ */
+ public String getCatalog() {
+ return catalog;
+ }
+
+ /**
+ * Retrieves the db schema from the object.
+ * @return the dbSchema
+ */
+ public String getDbSchema() {
+ return dbSchema;
+ }
+
+ /**
+ * Retreives the table from the object.
+ * @return the table.
+ */
+ public String getTable() {
+ return table;
+ }
+}
+
diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSql.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSql.java
new file mode 100644
index 0000000000000..159ef72401f7e
--- /dev/null
+++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSql.java
@@ -0,0 +1,706 @@
+/*
+ * 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.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.channels.Channels;
+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.stream.IntStream;
+
+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.flight.sql.impl.FlightSql;
+import org.apache.arrow.flight.sql.impl.FlightSql.SqlSupportedCaseSensitivity;
+import org.apache.arrow.flight.sql.util.TableRef;
+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.ReadChannel;
+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> 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> EXPECTED_RESULTS_FOR_PARAMETER_BINDING = ImmutableList.of(
+ asList("1", "one", "1", "1"));
+ private static final Map GET_SQL_INFO_EXPECTED_RESULTS_MAP = new LinkedHashMap<>();
+ private static final String LOCALHOST = "localhost";
+ private static BufferAllocator allocator;
+ private static FlightServer server;
+ private static FlightSqlClient sqlClient;
+ @Rule
+ public final ErrorCollector collector = new ErrorCollector();
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ allocator = new RootAllocator(Integer.MAX_VALUE);
+
+ final Location serverLocation = Location.forGrpcInsecure(LOCALHOST, 0);
+ server = FlightServer.builder(allocator, serverLocation, new FlightSqlExample(serverLocation))
+ .build()
+ .start();
+
+ final Location clientLocation = Location.forGrpcInsecure(LOCALHOST, server.getPort());
+ sqlClient = new FlightSqlClient(FlightClient.builder(allocator, clientLocation).build());
+
+ GET_SQL_INFO_EXPECTED_RESULTS_MAP
+ .put(Integer.toString(FlightSql.SqlInfo.FLIGHT_SQL_SERVER_NAME_VALUE), "Apache Derby");
+ GET_SQL_INFO_EXPECTED_RESULTS_MAP
+ .put(Integer.toString(FlightSql.SqlInfo.FLIGHT_SQL_SERVER_VERSION_VALUE), "10.14.2.0 - (1828579)");
+ GET_SQL_INFO_EXPECTED_RESULTS_MAP
+ .put(Integer.toString(FlightSql.SqlInfo.FLIGHT_SQL_SERVER_ARROW_VERSION_VALUE), "10.14.2.0 - (1828579)");
+ GET_SQL_INFO_EXPECTED_RESULTS_MAP
+ .put(Integer.toString(FlightSql.SqlInfo.FLIGHT_SQL_SERVER_READ_ONLY_VALUE), "false");
+ GET_SQL_INFO_EXPECTED_RESULTS_MAP
+ .put(Integer.toString(FlightSql.SqlInfo.SQL_DDL_CATALOG_VALUE), "false");
+ GET_SQL_INFO_EXPECTED_RESULTS_MAP
+ .put(Integer.toString(FlightSql.SqlInfo.SQL_DDL_SCHEMA_VALUE), "true");
+ GET_SQL_INFO_EXPECTED_RESULTS_MAP
+ .put(Integer.toString(FlightSql.SqlInfo.SQL_DDL_TABLE_VALUE), "true");
+ GET_SQL_INFO_EXPECTED_RESULTS_MAP
+ .put(
+ Integer.toString(FlightSql.SqlInfo.SQL_IDENTIFIER_CASE_VALUE),
+ Integer.toString(SqlSupportedCaseSensitivity.SQL_CASE_SENSITIVITY_UPPERCASE_VALUE));
+ GET_SQL_INFO_EXPECTED_RESULTS_MAP
+ .put(Integer.toString(FlightSql.SqlInfo.SQL_IDENTIFIER_QUOTE_CHAR_VALUE), "\"");
+ GET_SQL_INFO_EXPECTED_RESULTS_MAP
+ .put(
+ Integer.toString(FlightSql.SqlInfo.SQL_QUOTED_IDENTIFIER_CASE_VALUE),
+ Integer.toString(SqlSupportedCaseSensitivity.SQL_CASE_SENSITIVITY_CASE_INSENSITIVE_VALUE));
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ close(sqlClient, server, allocator);
+ }
+
+ private static List> getNonConformingResultsForGetSqlInfo(final List extends List> results) {
+ return getNonConformingResultsForGetSqlInfo(results,
+ FlightSql.SqlInfo.FLIGHT_SQL_SERVER_NAME,
+ FlightSql.SqlInfo.FLIGHT_SQL_SERVER_VERSION,
+ FlightSql.SqlInfo.FLIGHT_SQL_SERVER_ARROW_VERSION,
+ FlightSql.SqlInfo.FLIGHT_SQL_SERVER_READ_ONLY,
+ FlightSql.SqlInfo.SQL_DDL_CATALOG,
+ FlightSql.SqlInfo.SQL_DDL_SCHEMA,
+ FlightSql.SqlInfo.SQL_DDL_TABLE,
+ FlightSql.SqlInfo.SQL_IDENTIFIER_CASE,
+ FlightSql.SqlInfo.SQL_IDENTIFIER_QUOTE_CHAR,
+ FlightSql.SqlInfo.SQL_QUOTED_IDENTIFIER_CASE);
+ }
+
+ private static List> getNonConformingResultsForGetSqlInfo(
+ final List extends List> results,
+ final FlightSql.SqlInfo... args) {
+ final List> nonConformingResults = new ArrayList<>();
+ if (results.size() == args.length) {
+ for (int index = 0; index < results.size(); index++) {
+ final List result = results.get(index);
+ final String providedName = result.get(0);
+ final String expectedName = Integer.toString(args[index].getNumber());
+ if (!(GET_SQL_INFO_EXPECTED_RESULTS_MAP.get(providedName).equals(result.get(1)) &&
+ providedName.equals(expectedName))) {
+ nonConformingResults.add(result);
+ break;
+ }
+ }
+ }
+ return nonConformingResults;
+ }
+
+ @Test
+ public void testGetTablesSchema() {
+ final FlightInfo info = sqlClient.getTables(null, null, null, null, true);
+ collector.checkThat(info.getSchema(), is(FlightSqlProducer.Schemas.GET_TABLES_SCHEMA));
+ }
+
+ @Test
+ public void testGetTablesResultNoSchema() throws Exception {
+ try (final FlightStream stream =
+ sqlClient.getStream(
+ sqlClient.getTables(null, null, null, null, false)
+ .getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(FlightSqlProducer.Schemas.GET_TABLES_SCHEMA_NO_SCHEMA));
+ final List> results = getResults(stream);
+ final List> expectedResults = ImmutableList.of(
+ // catalog_name | schema_name | table_name | table_type | table_schema
+ asList(null /* TODO No catalog yet */, "SYS", "SYSALIASES", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSCHECKS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSCOLPERMS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSCOLUMNS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSCONGLOMERATES", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSCONSTRAINTS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSDEPENDS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSFILES", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSFOREIGNKEYS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSKEYS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSPERMS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSROLES", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSROUTINEPERMS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSSCHEMAS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSSEQUENCES", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSSTATEMENTS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSSTATISTICS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSTABLEPERMS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSTABLES", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSTRIGGERS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSUSERS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYS", "SYSVIEWS", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "SYSIBM", "SYSDUMMY1", "SYSTEM TABLE"),
+ asList(null /* TODO No catalog yet */, "APP", "FOREIGNTABLE", "TABLE"),
+ asList(null /* TODO No catalog yet */, "APP", "INTTABLE", "TABLE"));
+ collector.checkThat(results, is(expectedResults));
+ }
+ }
+
+ @Test
+ public void testGetTablesResultFilteredNoSchema() throws Exception {
+ try (final FlightStream stream =
+ sqlClient.getStream(
+ sqlClient.getTables(null, null, null, singletonList("TABLE"), false)
+ .getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(FlightSqlProducer.Schemas.GET_TABLES_SCHEMA_NO_SCHEMA));
+ final List> results = getResults(stream);
+ final List> expectedResults = ImmutableList.of(
+ // catalog_name | schema_name | table_name | table_type | table_schema
+ asList(null /* TODO No catalog yet */, "APP", "FOREIGNTABLE", "TABLE"),
+ asList(null /* TODO No catalog yet */, "APP", "INTTABLE", "TABLE"));
+ collector.checkThat(results, is(expectedResults));
+ }
+ }
+
+ @Test
+ public void testGetTablesResultFilteredWithSchema() throws Exception {
+ try (final FlightStream stream =
+ sqlClient.getStream(
+ sqlClient.getTables(null, null, null, singletonList("TABLE"), true)
+ .getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(FlightSqlProducer.Schemas.GET_TABLES_SCHEMA));
+ final List> results = getResults(stream);
+ final List> expectedResults = ImmutableList.of(
+ // catalog_name | schema_name | table_name | table_type | table_schema
+ asList(
+ null /* TODO No catalog yet */,
+ "APP",
+ "FOREIGNTABLE",
+ "TABLE",
+ new Schema(asList(
+ new Field("ID", new FieldType(false, MinorType.INT.getType(), null), null),
+ Field.nullable("FOREIGNNAME", MinorType.VARCHAR.getType()),
+ Field.nullable("VALUE", MinorType.INT.getType()))).toJson()),
+ asList(
+ null /* TODO No catalog yet */,
+ "APP",
+ "INTTABLE",
+ "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()))).toJson()));
+ collector.checkThat(results, is(expectedResults));
+ }
+ }
+
+ @Test
+ public void testSimplePreparedStatementSchema() throws Exception {
+ try (final PreparedStatement preparedStatement = sqlClient.prepare("SELECT * FROM intTable")) {
+ final Schema actualSchema = preparedStatement.getResultSetSchema();
+ collector.checkThat(actualSchema, is(SCHEMA_INT_TABLE));
+
+ final FlightInfo info = preparedStatement.execute();
+ collector.checkThat(info.getSchema(), is(SCHEMA_INT_TABLE));
+ }
+ }
+
+ @Test
+ public void testSimplePreparedStatementResults() throws Exception {
+ try (final PreparedStatement preparedStatement = sqlClient.prepare("SELECT * FROM intTable");
+ final FlightStream stream = sqlClient.getStream(
+ preparedStatement.execute().getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(SCHEMA_INT_TABLE));
+ collector.checkThat(getResults(stream), is(EXPECTED_RESULTS_FOR_STAR_SELECT_QUERY));
+ }
+ }
+
+ @Test
+ public void testSimplePreparedStatementResultsWithParameterBinding() throws Exception {
+ try (PreparedStatement prepare = sqlClient.prepare("SELECT * FROM intTable WHERE id = ?")) {
+ final Schema parameterSchema = prepare.getParameterSchema();
+ try (final VectorSchemaRoot insertRoot = VectorSchemaRoot.create(parameterSchema, allocator)) {
+ insertRoot.allocateNew();
+
+ final IntVector valueVector = (IntVector) insertRoot.getVector(0);
+ valueVector.setSafe(0, 1);
+ insertRoot.setRowCount(1);
+
+ prepare.setParameters(insertRoot);
+ FlightInfo flightInfo = prepare.execute();
+
+ FlightStream stream = sqlClient.getStream(flightInfo
+ .getEndpoints()
+ .get(0).getTicket());
+
+ collector.checkThat(stream.getSchema(), is(SCHEMA_INT_TABLE));
+ collector.checkThat(getResults(stream), is(EXPECTED_RESULTS_FOR_PARAMETER_BINDING));
+ }
+ }
+ }
+
+ @Test
+ public void testSimplePreparedStatementUpdateResults() throws SQLException {
+ try (PreparedStatement prepare = sqlClient.prepare("INSERT INTO INTTABLE (keyName, value ) VALUES (?, ?)");
+ PreparedStatement deletePrepare = sqlClient.prepare("DELETE FROM INTTABLE WHERE keyName = ?")) {
+ final Schema parameterSchema = prepare.getParameterSchema();
+ try (final VectorSchemaRoot insertRoot = VectorSchemaRoot.create(parameterSchema, allocator)) {
+ final VarCharVector varCharVector = (VarCharVector) insertRoot.getVector(0);
+ final IntVector valueVector = (IntVector) insertRoot.getVector(1);
+ final int counter = 10;
+ insertRoot.allocateNew();
+
+ final IntStream range = IntStream.range(0, counter);
+
+ range.forEach(i -> {
+ valueVector.setSafe(i, i * counter);
+ varCharVector.setSafe(i, new Text("value" + i));
+ });
+
+ insertRoot.setRowCount(counter);
+
+ prepare.setParameters(insertRoot);
+ final long updatedRows = prepare.executeUpdate();
+
+ final long deletedRows;
+ try (final VectorSchemaRoot deleteRoot = VectorSchemaRoot.of(varCharVector)) {
+ deletePrepare.setParameters(deleteRoot);
+ deletedRows = deletePrepare.executeUpdate();
+ }
+
+ collector.checkThat(updatedRows, is(10L));
+ collector.checkThat(deletedRows, is(10L));
+ }
+ }
+ }
+
+ @Test
+ public void testSimplePreparedStatementUpdateResultsWithoutParameters() throws SQLException {
+ try (PreparedStatement prepare = sqlClient
+ .prepare("INSERT INTO INTTABLE (keyName, value ) VALUES ('test', 1000)");
+ PreparedStatement deletePrepare = sqlClient.prepare("DELETE FROM INTTABLE WHERE keyName = 'test'")) {
+ final long updatedRows = prepare.executeUpdate();
+
+ final long deletedRows = deletePrepare.executeUpdate();
+
+ collector.checkThat(updatedRows, is(1L));
+ collector.checkThat(deletedRows, is(1L));
+ }
+ }
+
+ @Test
+ public void testSimplePreparedStatementClosesProperly() {
+ final PreparedStatement preparedStatement = sqlClient.prepare("SELECT * FROM intTable");
+ collector.checkThat(preparedStatement.isClosed(), is(false));
+ preparedStatement.close();
+ collector.checkThat(preparedStatement.isClosed(), is(true));
+ }
+
+ @Test
+ public void testGetCatalogsSchema() {
+ final FlightInfo info = sqlClient.getCatalogs();
+ collector.checkThat(info.getSchema(), is(FlightSqlProducer.Schemas.GET_CATALOGS_SCHEMA));
+ }
+
+ @Test
+ public void testGetCatalogsResults() throws Exception {
+ try (final FlightStream stream =
+ sqlClient.getStream(sqlClient.getCatalogs().getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(FlightSqlProducer.Schemas.GET_CATALOGS_SCHEMA));
+ List> catalogs = getResults(stream);
+ collector.checkThat(catalogs, is(emptyList()));
+ }
+ }
+
+ @Test
+ public void testGetTableTypesSchema() {
+ final FlightInfo info = sqlClient.getTableTypes();
+ collector.checkThat(info.getSchema(), is(FlightSqlProducer.Schemas.GET_TABLE_TYPES_SCHEMA));
+ }
+
+ @Test
+ public void testGetTableTypesResult() throws Exception {
+ try (final FlightStream stream =
+ sqlClient.getStream(sqlClient.getTableTypes().getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(FlightSqlProducer.Schemas.GET_TABLE_TYPES_SCHEMA));
+ final List> tableTypes = getResults(stream);
+ final List> expectedTableTypes = ImmutableList.of(
+ // table_type
+ singletonList("SYNONYM"),
+ singletonList("SYSTEM TABLE"),
+ singletonList("TABLE"),
+ singletonList("VIEW")
+ );
+ collector.checkThat(tableTypes, is(expectedTableTypes));
+ }
+ }
+
+ @Test
+ public void testGetSchemasSchema() {
+ final FlightInfo info = sqlClient.getSchemas(null, null);
+ collector.checkThat(info.getSchema(), is(FlightSqlProducer.Schemas.GET_SCHEMAS_SCHEMA));
+ }
+
+ @Test
+ public void testGetSchemasResult() throws Exception {
+ try (final FlightStream stream =
+ sqlClient.getStream(sqlClient.getSchemas(null, null).getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(FlightSqlProducer.Schemas.GET_SCHEMAS_SCHEMA));
+ final List> schemas = getResults(stream);
+ final List> expectedSchemas = ImmutableList.of(
+ // catalog_name | schema_name
+ asList(null /* TODO Add catalog. */, "APP"),
+ asList(null /* TODO Add catalog. */, "NULLID"),
+ asList(null /* TODO Add catalog. */, "SQLJ"),
+ asList(null /* TODO Add catalog. */, "SYS"),
+ asList(null /* TODO Add catalog. */, "SYSCAT"),
+ asList(null /* TODO Add catalog. */, "SYSCS_DIAG"),
+ asList(null /* TODO Add catalog. */, "SYSCS_UTIL"),
+ asList(null /* TODO Add catalog. */, "SYSFUN"),
+ asList(null /* TODO Add catalog. */, "SYSIBM"),
+ asList(null /* TODO Add catalog. */, "SYSPROC"),
+ asList(null /* TODO Add catalog. */, "SYSSTAT"));
+ collector.checkThat(schemas, is(expectedSchemas));
+ }
+ }
+
+ @Test
+ public void testGetPrimaryKey() {
+ final FlightInfo flightInfo = sqlClient.getPrimaryKeys(TableRef.of(null, null, "INTTABLE"));
+ final FlightStream stream = sqlClient.getStream(flightInfo.getEndpoints().get(0).getTicket());
+
+ final List> results = getResults(stream);
+ collector.checkThat(results.size(), is(1));
+
+ final List result = results.get(0);
+
+ collector.checkThat(result.get(0), is(""));
+ collector.checkThat(result.get(1), is("APP"));
+ collector.checkThat(result.get(2), is("INTTABLE"));
+ collector.checkThat(result.get(3), is("ID"));
+ collector.checkThat(result.get(4), is("1"));
+ collector.checkThat(result.get(5), notNullValue());
+ }
+
+ @Test
+ public void testGetSqlInfoSchema() {
+ final FlightInfo info = sqlClient.getSqlInfo();
+ collector.checkThat(info.getSchema(), is(FlightSqlProducer.Schemas.GET_SQL_INFO_SCHEMA));
+ }
+
+ @Test
+ public void testGetSqlInfoResults() throws Exception {
+ final FlightInfo info = sqlClient.getSqlInfo();
+ try (final FlightStream stream = sqlClient.getStream(info.getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(FlightSqlProducer.Schemas.GET_SQL_INFO_SCHEMA));
+ collector.checkThat(getNonConformingResultsForGetSqlInfo(getResults(stream)), is(emptyList()));
+ }
+ }
+
+ @Test
+ public void testGetSqlInfoResultsWithSingleArg() throws Exception {
+ final FlightSql.SqlInfo arg = FlightSql.SqlInfo.FLIGHT_SQL_SERVER_NAME;
+ final FlightInfo info = sqlClient.getSqlInfo(arg);
+ try (final FlightStream stream = sqlClient.getStream(info.getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(FlightSqlProducer.Schemas.GET_SQL_INFO_SCHEMA));
+ collector.checkThat(getNonConformingResultsForGetSqlInfo(getResults(stream), arg), is(emptyList()));
+ }
+ }
+
+ @Test
+ public void testGetSqlInfoResultsWithTwoArgs() throws Exception {
+ final FlightSql.SqlInfo[] args = {
+ FlightSql.SqlInfo.FLIGHT_SQL_SERVER_NAME,
+ FlightSql.SqlInfo.FLIGHT_SQL_SERVER_VERSION};
+ final FlightInfo info = sqlClient.getSqlInfo(args);
+ try (final FlightStream stream = sqlClient.getStream(info.getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(FlightSqlProducer.Schemas.GET_SQL_INFO_SCHEMA));
+ collector.checkThat(getNonConformingResultsForGetSqlInfo(getResults(stream), args), is(emptyList()));
+ }
+ }
+
+ @Test
+ public void testGetSqlInfoResultsWithThreeArgs() throws Exception {
+ final FlightSql.SqlInfo[] args = {
+ FlightSql.SqlInfo.FLIGHT_SQL_SERVER_NAME,
+ FlightSql.SqlInfo.FLIGHT_SQL_SERVER_VERSION,
+ FlightSql.SqlInfo.SQL_IDENTIFIER_QUOTE_CHAR};
+ final FlightInfo info = sqlClient.getSqlInfo(args);
+ try (final FlightStream stream = sqlClient.getStream(info.getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(FlightSqlProducer.Schemas.GET_SQL_INFO_SCHEMA));
+ collector.checkThat(getNonConformingResultsForGetSqlInfo(getResults(stream), args), is(emptyList()));
+ }
+ }
+
+ @Test
+ public void testGetCommandExportedKeys() {
+ final FlightStream stream =
+ sqlClient.getStream(
+ sqlClient.getExportedKeys(TableRef.of(null, null, "FOREIGNTABLE"))
+ .getEndpoints().get(0).getTicket());
+
+ final List> results = getResults(stream);
+
+ final List> matchers = asList(
+ nullValue(String.class), // pk_catalog_name
+ is("APP"), // pk_schema_name
+ is("FOREIGNTABLE"), // pk_table_name
+ is("ID"), // pk_column_name
+ nullValue(String.class), // fk_catalog_name
+ is("APP"), // fk_schema_name
+ is("INTTABLE"), // fk_table_name
+ is("FOREIGNID"), // fk_column_name
+ is("1"), // key_sequence
+ containsString("SQL"), // fk_key_name
+ containsString("SQL"), // pk_key_name
+ is("3"), // update_rule
+ is("3")); // delete_rule
+
+ Assert.assertEquals(1, results.size());
+ for (int i = 0; i < matchers.size(); i++) {
+ collector.checkThat(results.get(0).get(i), matchers.get(i));
+ }
+ }
+
+ @Test
+ public void testGetCommandImportedKeys() {
+ final FlightStream stream =
+ sqlClient.getStream(
+ sqlClient.getImportedKeys(TableRef.of(null, null, "INTTABLE"))
+ .getEndpoints().get(0).getTicket());
+
+ final List> results = getResults(stream);
+
+ final List> matchers = asList(
+ nullValue(String.class), // pk_catalog_name
+ is("APP"), // pk_schema_name
+ is("FOREIGNTABLE"), // pk_table_name
+ is("ID"), // pk_column_name
+ nullValue(String.class), // fk_catalog_name
+ is("APP"), // fk_schema_name
+ is("INTTABLE"), // fk_table_name
+ is("FOREIGNID"), // fk_column_name
+ is("1"), // key_sequence
+ containsString("SQL"), // fk_key_name
+ containsString("SQL"), // pk_key_name
+ is("3"), // update_rule
+ is("3")); // delete_rule
+
+ Assert.assertEquals(1, results.size());
+ for (int i = 0; i < matchers.size(); i++) {
+ collector.checkThat(results.get(0).get(i), matchers.get(i));
+ }
+ }
+
+ @Test
+ public void testGetCommandCrossReference() {
+ final FlightInfo flightInfo = sqlClient.getCrossReference(TableRef.of(null, null,
+ "FOREIGNTABLE"), TableRef.of(null, null, "INTTABLE"));
+ final FlightStream stream = sqlClient.getStream(flightInfo.getEndpoints().get(0).getTicket());
+
+ final List> results = getResults(stream);
+
+ final List> matchers = asList(
+ nullValue(String.class), // pk_catalog_name
+ is("APP"), // pk_schema_name
+ is("FOREIGNTABLE"), // pk_table_name
+ is("ID"), // pk_column_name
+ nullValue(String.class), // fk_catalog_name
+ is("APP"), // fk_schema_name
+ is("INTTABLE"), // fk_table_name
+ is("FOREIGNID"), // fk_column_name
+ is("1"), // key_sequence
+ containsString("SQL"), // fk_key_name
+ containsString("SQL"), // pk_key_name
+ is("3"), // update_rule
+ is("3")); // delete_rule
+
+ Assert.assertEquals(1, results.size());
+ for (int i = 0; i < matchers.size(); i++) {
+ collector.checkThat(results.get(0).get(i), matchers.get(i));
+ }
+ }
+
+ @Test
+ public void testCreateStatementSchema() throws Exception {
+ final FlightInfo info = sqlClient.execute("SELECT * FROM intTable");
+ collector.checkThat(info.getSchema(), is(SCHEMA_INT_TABLE));
+
+ // Consume statement to close connection before cache eviction
+ try (FlightStream stream = sqlClient.getStream(info.getEndpoints().get(0).getTicket())) {
+ while (stream.next()) {
+ // Do nothing
+ }
+ }
+ }
+
+ @Test
+ public void testCreateStatementResults() throws Exception {
+ try (final FlightStream stream = sqlClient
+ .getStream(sqlClient.execute("SELECT * FROM intTable").getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(SCHEMA_INT_TABLE));
+ collector.checkThat(getResults(stream), is(EXPECTED_RESULTS_FOR_STAR_SELECT_QUERY));
+ }
+ }
+
+ List> getResults(FlightStream stream) {
+ final List> results = new ArrayList<>();
+ while (stream.next()) {
+ try (final VectorSchemaRoot root = stream.getRoot()) {
+ final long rowCount = root.getRowCount();
+ for (int i = 0; i < rowCount; ++i) {
+ results.add(new ArrayList<>());
+ }
+
+ root.getSchema().getFields().forEach(field -> {
+ try (final FieldVector fieldVector = root.getVector(field.getName())) {
+ if (fieldVector instanceof VarCharVector) {
+ final VarCharVector varcharVector = (VarCharVector) fieldVector;
+ for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) {
+ final Text data = varcharVector.getObject(rowIndex);
+ results.get(rowIndex).add(isNull(data) ? null : data.toString());
+ }
+ } else if (fieldVector instanceof IntVector) {
+ for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) {
+ results.get(rowIndex).add(String.valueOf(((IntVector) fieldVector).get(rowIndex)));
+ }
+ } else if (fieldVector instanceof VarBinaryVector) {
+ final VarBinaryVector varbinaryVector = (VarBinaryVector) fieldVector;
+ for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) {
+ final byte[] data = varbinaryVector.getObject(rowIndex);
+ final String output;
+ try {
+ output = isNull(data) ?
+ null :
+ MessageSerializer.deserializeSchema(
+ new ReadChannel(Channels.newChannel(new ByteArrayInputStream(data)))).toJson();
+ } catch (final IOException e) {
+ throw new RuntimeException("Failed to deserialize schema", e);
+ }
+ results.get(rowIndex).add(output);
+ }
+ } else if (fieldVector instanceof DenseUnionVector) {
+ final DenseUnionVector denseUnionVector = (DenseUnionVector) fieldVector;
+ for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) {
+ final Object data = denseUnionVector.getObject(rowIndex);
+ results.get(rowIndex).add(isNull(data) ? null : Objects.toString(data));
+ }
+ } else if (fieldVector instanceof UInt4Vector) {
+ final UInt4Vector uInt4Vector = (UInt4Vector) fieldVector;
+ for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) {
+ final Object data = uInt4Vector.getObject(rowIndex);
+ results.get(rowIndex).add(isNull(data) ? null : Objects.toString(data));
+ }
+ } else if (fieldVector instanceof UInt1Vector) {
+ final UInt1Vector uInt1Vector = (UInt1Vector) fieldVector;
+ for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) {
+ final Object data = uInt1Vector.getObject(rowIndex);
+ results.get(rowIndex).add(isNull(data) ? null : Objects.toString(data));
+ }
+ } else {
+ throw new UnsupportedOperationException("Not yet implemented");
+ }
+ }
+ });
+ }
+ }
+
+ return results;
+ }
+
+ @Test
+ public void testExecuteUpdate() {
+ long insertedCount = sqlClient.executeUpdate("INSERT INTO INTTABLE (keyName, value) VALUES " +
+ "('KEYNAME1', 1001), ('KEYNAME2', 1002), ('KEYNAME3', 1003)");
+ collector.checkThat(insertedCount, is(3L));
+
+ long updatedCount = sqlClient.executeUpdate("UPDATE INTTABLE SET keyName = 'KEYNAME1' " +
+ "WHERE keyName = 'KEYNAME2' OR keyName = 'KEYNAME3'");
+ collector.checkThat(updatedCount, is(2L));
+
+ long deletedCount = sqlClient.executeUpdate("DELETE FROM INTTABLE WHERE keyName = 'KEYNAME1'");
+ collector.checkThat(deletedCount, is(3L));
+ }
+
+ @Test
+ public void testQueryWithNoResultsShouldNotHang() throws Exception {
+ try (final PreparedStatement preparedStatement = sqlClient.prepare("SELECT * FROM intTable WHERE 1 = 0");
+ final FlightStream stream = sqlClient
+ .getStream(preparedStatement.execute().getEndpoints().get(0).getTicket())) {
+ collector.checkThat(stream.getSchema(), is(SCHEMA_INT_TABLE));
+
+ final List> result = getResults(stream);
+ collector.checkThat(result, is(emptyList()));
+ }
+ }
+}
diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlExample.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlExample.java
new file mode 100644
index 0000000000000..687840386e960
--- /dev/null
+++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlExample.java
@@ -0,0 +1,1622 @@
+/*
+ * 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 org.apache.arrow.adapter.jdbc.JdbcToArrow.sqlToArrowVectorIterator;
+import static org.apache.arrow.adapter.jdbc.JdbcToArrowUtils.jdbcToArrowSchema;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetCrossReference;
+import static org.apache.arrow.flight.sql.impl.FlightSql.CommandGetDbSchemas;
+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.DoPutUpdateResult;
+import static org.apache.arrow.flight.sql.impl.FlightSql.TicketStatementQuery;
+import static org.apache.arrow.util.Preconditions.checkState;
+import static org.slf4j.LoggerFactory.getLogger;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+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.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.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.SqlInfoBuilder;
+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.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.flight.sql.impl.FlightSql.SqlSupportedCaseSensitivity;
+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.ipc.WriteChannel;
+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.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:
+ *
+ * - 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 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 dataSource;
+ private final BufferAllocator rootAllocator = new RootAllocator();
+ private final Cache> preparedStatementLoadingCache;
+ private final Cache> statementLoadingCache;
+ private final SqlInfoBuilder sqlInfoBuilder;
+
+ 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 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())
+ .build();
+
+ statementLoadingCache =
+ CacheBuilder.newBuilder()
+ .maximumSize(100)
+ .expireAfterWrite(10, TimeUnit.MINUTES)
+ .removalListener(new StatementRemovalListener<>())
+ .build();
+
+ this.location = location;
+
+ sqlInfoBuilder = new SqlInfoBuilder();
+ try (final Connection connection = dataSource.getConnection()) {
+ final DatabaseMetaData metaData = connection.getMetaData();
+
+ sqlInfoBuilder.withFlightSqlServerName(metaData.getDatabaseProductName())
+ .withFlightSqlServerVersion(metaData.getDatabaseProductVersion())
+ .withFlightSqlServerArrowVersion(metaData.getDriverVersion())
+ .withFlightSqlServerReadOnly(metaData.isReadOnly())
+ .withSqlIdentifierQuoteChar(metaData.getIdentifierQuoteString())
+ .withSqlDdlCatalog(metaData.supportsCatalogsInDataManipulation())
+ .withSqlDdlSchema( metaData.supportsSchemasInDataManipulation())
+ .withSqlDdlTable( metaData.allTablesAreSelectable())
+ .withSqlIdentifierCase(metaData.storesMixedCaseIdentifiers() ?
+ SqlSupportedCaseSensitivity.SQL_CASE_SENSITIVITY_CASE_INSENSITIVE :
+ metaData.storesUpperCaseIdentifiers() ?
+ SqlSupportedCaseSensitivity.SQL_CASE_SENSITIVITY_UPPERCASE :
+ metaData.storesLowerCaseIdentifiers() ?
+ SqlSupportedCaseSensitivity.SQL_CASE_SENSITIVITY_LOWERCASE :
+ SqlSupportedCaseSensitivity.SQL_CASE_SENSITIVITY_UNKNOWN)
+ .withSqlQuotedIdentifierCase(metaData.storesMixedCaseQuotedIdentifiers() ?
+ SqlSupportedCaseSensitivity.SQL_CASE_SENSITIVITY_CASE_INSENSITIVE :
+ metaData.storesUpperCaseQuotedIdentifiers() ?
+ SqlSupportedCaseSensitivity.SQL_CASE_SENSITIVITY_UPPERCASE :
+ metaData.storesLowerCaseQuotedIdentifiers() ?
+ SqlSupportedCaseSensitivity.SQL_CASE_SENSITIVITY_LOWERCASE :
+ SqlSupportedCaseSensitivity.SQL_CASE_SENSITIVITY_UNKNOWN);
+ } catch (SQLException e) {
+ throw new RuntimeException(e);
+ }
+
+ }
+
+ private static boolean removeDerbyDatabaseIfExists() {
+ boolean wasSuccess;
+ final Path path = Paths.get("target" + File.separator + "derbyDB");
+
+ try (final Stream 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` 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 =
+ JdbcToArrowUtils.getArrowTypeFromJdbcType(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 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 void vectorConsumer(final T data, final V vector,
+ final Consumer consumerIfNullable,
+ final BiConsumer 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", FieldType.notNullable(MinorType.VARCHAR.getType()), allocator);
+ final List vectors = ImmutableList.of(catalogs, schemas);
+ vectors.forEach(FieldVector::allocateNew);
+ final Map 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 int saveToVectors(final Map vectorToColumnName,
+ final ResultSet data, boolean emptyToNull)
+ throws SQLException {
+ Objects.requireNonNull(vectorToColumnName, "vectorToColumnName cannot be null.");
+ Objects.requireNonNull(data, "data cannot be null.");
+ final Set> entrySet = vectorToColumnName.entrySet();
+ int rows = 0;
+ for (; data.next(); rows++) {
+ for (final Entry 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 vectorToColumn : entrySet) {
+ vectorToColumn.getKey().setValueCount(rows);
+ }
+
+ return rows;
+ }
+
+ private static void saveToVectors(final Map 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, FieldType.notNullable(MinorType.VARCHAR.getType()), 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", FieldType.notNullable(MinorType.VARCHAR.getType()), allocator);
+ final VarCharVector tableTypeVector =
+ new VarCharVector("table_type", FieldType.notNullable(MinorType.VARCHAR.getType()), allocator);
+
+ final List vectors = new ArrayList<>(4);
+ vectors.add(catalogNameVector);
+ vectors.add(schemaNameVector);
+ vectors.add(tableNameVector);
+ vectors.add(tableTypeVector);
+
+ vectors.forEach(FieldVector::allocateNew);
+
+ final Map vectorToColumnName = ImmutableMap.of(
+ catalogNameVector, "TABLE_CAT",
+ schemaNameVector, "TABLE_SCHEM",
+ tableNameVector, "TABLE_NAME",
+ tableTypeVector, "TABLE_TYPE");
+
+ try (final ResultSet data =
+ Objects.requireNonNull(
+ databaseMetaData,
+ format("%s cannot be null.", databaseMetaData.getClass().getName()))
+ .getTables(catalog, schemaFilterPattern, tableFilterPattern, tableTypes)) {
+
+ saveToVectors(vectorToColumnName, data, true);
+ final int rows =
+ vectors.stream().map(FieldVector::getValueCount).findAny().orElseThrow(IllegalStateException::new);
+ vectors.forEach(vector -> vector.setValueCount(rows));
+
+ if (includeSchema) {
+ final VarBinaryVector tableSchemaVector =
+ new VarBinaryVector("table_schema", FieldType.notNullable(MinorType.VARBINARY.getType()), allocator);
+ tableSchemaVector.allocateNew(rows);
+
+ try (final ResultSet columnsData =
+ databaseMetaData.getColumns(catalog, schemaFilterPattern, tableFilterPattern, null)) {
+ final Map> tableToFields = new HashMap<>();
+
+ while (columnsData.next()) {
+ final String tableName = columnsData.getString("TABLE_NAME");
+ final String fieldName = columnsData.getString("COLUMN_NAME");
+ final int dataType = columnsData.getInt("DATA_TYPE");
+ final boolean isNullable = columnsData.getInt("NULLABLE") != DatabaseMetaData.columnNoNulls;
+ final int precision = columnsData.getInt("NUM_PREC_RADIX");
+ final int scale = columnsData.getInt("DECIMAL_DIGITS");
+ final List fields = tableToFields.computeIfAbsent(tableName, tableName_ -> new ArrayList<>());
+ final Field field =
+ new Field(
+ fieldName,
+ new FieldType(
+ isNullable,
+ getArrowTypeFromJdbcType(dataType, precision, scale),
+ null),
+ null);
+ fields.add(field);
+ }
+
+ for (int index = 0; index < rows; index++) {
+ final String tableName = tableNameVector.getObject(index).toString();
+ final Schema schema = new Schema(tableToFields.get(tableName));
+ saveToVector(
+ copyFrom(serializeMetadata(schema)).toByteArray(),
+ tableSchemaVector, index);
+ }
+ }
+
+ tableSchemaVector.setValueCount(rows);
+ vectors.add(tableSchemaVector);
+ }
+ }
+
+ return new VectorSchemaRoot(vectors);
+ }
+
+ private static ByteBuffer serializeMetadata(final Schema schema) {
+ final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+ try {
+ MessageSerializer.serialize(new WriteChannel(Channels.newChannel(outputStream)), schema);
+
+ return ByteBuffer.wrap(outputStream.toByteArray());
+ } catch (final IOException e) {
+ throw new RuntimeException("Failed to serialize schema", e);
+ }
+ }
+
+ @Override
+ public void getStreamPreparedStatement(final CommandPreparedStatementQuery command, final CallContext context,
+ final ServerStreamListener listener) {
+ final ByteString handle = command.getPreparedStatementHandle();
+ StatementContext statementContext = preparedStatementLoadingCache.getIfPresent(handle);
+ Objects.requireNonNull(statementContext);
+ final PreparedStatement statement = statementContext.getStatement();
+ try (final ResultSet resultSet = statement.executeQuery()) {
+ final Schema schema = jdbcToArrowSchema(resultSet.getMetaData(), DEFAULT_CALENDAR);
+ try (final VectorSchemaRoot vectorSchemaRoot = VectorSchemaRoot.create(schema, rootAllocator)) {
+ final VectorLoader loader = new VectorLoader(vectorSchemaRoot);
+ listener.start(vectorSchemaRoot);
+
+ final ArrowVectorIterator iterator = sqlToArrowVectorIterator(resultSet, rootAllocator);
+ while (iterator.hasNext()) {
+ final VectorSchemaRoot batch = iterator.next();
+ if (batch.getRowCount() == 0) {
+ break;
+ }
+ final VectorUnloader unloader = new VectorUnloader(batch);
+ loader.load(unloader.getRecordBatch());
+ listener.putNext();
+ vectorSchemaRoot.clear();
+ }
+
+ listener.putNext();
+ }
+ } catch (final SQLException | IOException e) {
+ LOGGER.error(format("Failed to getStreamPreparedStatement: <%s>.", e.getMessage()), e);
+ listener.error(e);
+ } finally {
+ listener.completed();
+ }
+ }
+
+ @Override
+ public void closePreparedStatement(final ActionClosePreparedStatementRequest request, final CallContext context,
+ final StreamListener listener) {
+ try {
+ preparedStatementLoadingCache.invalidate(request.getPreparedStatementHandle());
+ } catch (final Exception e) {
+ listener.onError(e);
+ return;
+ }
+ listener.onCompleted();
+ }
+
+ @Override
+ public FlightInfo getFlightInfoStatement(final CommandStatementQuery request, final CallContext context,
+ final FlightDescriptor descriptor) {
+ ByteString handle = copyFrom(randomUUID().toString().getBytes(StandardCharsets.UTF_8));
+
+ try {
+ // Ownership of the connection will be passed to the context. Do NOT close!
+ final Connection connection = dataSource.getConnection();
+ final Statement statement = connection.createStatement(
+ ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_READ_ONLY);
+ final String query = request.getQuery();
+ final StatementContext statementContext = new StatementContext<>(statement, query);
+
+ statementLoadingCache.put(handle, statementContext);
+ final ResultSet resultSet = statement.executeQuery(query);
+
+ TicketStatementQuery ticket = TicketStatementQuery.newBuilder()
+ .setStatementHandle(handle)
+ .build();
+ return getFlightInfoForSchema(ticket, descriptor,
+ jdbcToArrowSchema(resultSet.getMetaData(), DEFAULT_CALENDAR));
+ } catch (final SQLException e) {
+ LOGGER.error(
+ format("There was a problem executing the prepared statement: <%s>.", e.getMessage()),
+ e);
+ throw CallStatus.INTERNAL.withCause(e).toRuntimeException();
+ }
+ }
+
+ @Override
+ public FlightInfo getFlightInfoPreparedStatement(final CommandPreparedStatementQuery command,
+ final CallContext context,
+ final FlightDescriptor descriptor) {
+ final ByteString preparedStatementHandle = command.getPreparedStatementHandle();
+ StatementContext statementContext =
+ preparedStatementLoadingCache.getIfPresent(preparedStatementHandle);
+ try {
+ assert statementContext != null;
+ PreparedStatement statement = statementContext.getStatement();
+
+ ResultSetMetaData metaData = statement.getMetaData();
+ return getFlightInfoForSchema(command, descriptor,
+ jdbcToArrowSchema(metaData, DEFAULT_CALENDAR));
+ } catch (final SQLException e) {
+ LOGGER.error(
+ format("There was a problem executing the prepared statement: <%s>.", e.getMessage()),
+ e);
+ throw CallStatus.INTERNAL.withCause(e).toRuntimeException();
+ }
+ }
+
+ @Override
+ public SchemaResult getSchemaStatement(final CommandStatementQuery command, final CallContext context,
+ final FlightDescriptor descriptor) {
+ throw CallStatus.UNIMPLEMENTED.toRuntimeException();
+ }
+
+ @Override
+ public void close() throws Exception {
+ try {
+ preparedStatementLoadingCache.cleanUp();
+ } catch (Throwable t) {
+ LOGGER.error(format("Failed to close resources: <%s>", t.getMessage()), t);
+ }
+
+ AutoCloseables.close(dataSource, rootAllocator);
+ }
+
+ @Override
+ public void listFlights(CallContext context, Criteria criteria, StreamListener listener) {
+ // TODO - build example implementation
+ throw CallStatus.UNIMPLEMENTED.toRuntimeException();
+ }
+
+ @Override
+ public void createPreparedStatement(final ActionCreatePreparedStatementRequest request, final CallContext context,
+ final StreamListener listener) {
+ try {
+ final ByteString preparedStatementHandle = copyFrom(randomUUID().toString().getBytes(StandardCharsets.UTF_8));
+ // Ownership of the connection will be passed to the context. Do NOT close!
+ final Connection connection = dataSource.getConnection();
+ final PreparedStatement preparedStatement = connection.prepareStatement(request.getQuery(),
+ ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_READ_ONLY);
+ final StatementContext preparedStatementContext =
+ new StatementContext<>(preparedStatement, request.getQuery());
+
+ preparedStatementLoadingCache.put(preparedStatementHandle, preparedStatementContext);
+
+ final Schema parameterSchema =
+ jdbcToArrowSchema(preparedStatement.getParameterMetaData(), DEFAULT_CALENDAR);
+
+ final ResultSetMetaData metaData = preparedStatement.getMetaData();
+ final ByteString bytes = isNull(metaData) ?
+ ByteString.EMPTY :
+ ByteString.copyFrom(
+ serializeMetadata(jdbcToArrowSchema(metaData, DEFAULT_CALENDAR)));
+ final ActionCreatePreparedStatementResult result = ActionCreatePreparedStatementResult.newBuilder()
+ .setDatasetSchema(bytes)
+ .setParameterSchema(copyFrom(serializeMetadata(parameterSchema)))
+ .setPreparedStatementHandle(preparedStatementHandle)
+ .build();
+ listener.onNext(new Result(pack(result).toByteArray()));
+ } catch (final Throwable t) {
+ listener.onError(t);
+ } finally {
+ listener.onCompleted();
+ }
+ }
+
+ @Override
+ public void doExchange(CallContext context, FlightStream reader, ServerStreamListener writer) {
+ // TODO - build example implementation
+ throw CallStatus.UNIMPLEMENTED.toRuntimeException();
+ }
+
+ @Override
+ public Runnable acceptPutStatement(CommandStatementUpdate command,
+ CallContext context, FlightStream flightStream,
+ StreamListener ackStream) {
+ final String query = command.getQuery();
+
+ return () -> {
+ try (final Connection connection = dataSource.getConnection();
+ final Statement statement = connection.createStatement()) {
+ final int result = statement.executeUpdate(query);
+
+ final DoPutUpdateResult build =
+ DoPutUpdateResult.newBuilder().setRecordCount(result).build();
+
+ try (final ArrowBuf buffer = rootAllocator.buffer(build.getSerializedSize())) {
+ buffer.writeBytes(build.toByteArray());
+ ackStream.onNext(PutResult.metadata(buffer));
+ ackStream.onCompleted();
+ }
+ } catch (SQLException e) {
+ ackStream.onError(e);
+ }
+ };
+ }
+
+ @Override
+ public Runnable acceptPutPreparedStatementUpdate(CommandPreparedStatementUpdate command, CallContext context,
+ FlightStream flightStream, StreamListener ackStream) {
+ final StatementContext statement =
+ preparedStatementLoadingCache.getIfPresent(command.getPreparedStatementHandle());
+
+ return () -> {
+ assert statement != null;
+ try {
+ final PreparedStatement preparedStatement = statement.getStatement();
+
+ while (flightStream.next()) {
+ final VectorSchemaRoot root = flightStream.getRoot();
+
+ final int rowCount = root.getRowCount();
+ final int recordCount;
+
+ if (rowCount == 0) {
+ preparedStatement.execute();
+ recordCount = preparedStatement.getUpdateCount();
+ } else {
+ setDataPreparedStatement(preparedStatement, root, true);
+ int[] recordCount1 = preparedStatement.executeBatch();
+ recordCount = Arrays.stream(recordCount1).sum();
+ }
+
+ final DoPutUpdateResult build =
+ DoPutUpdateResult.newBuilder().setRecordCount(recordCount).build();
+
+ try (final ArrowBuf buffer = rootAllocator.buffer(build.getSerializedSize())) {
+ buffer.writeBytes(build.toByteArray());
+ ackStream.onNext(PutResult.metadata(buffer));
+ }
+ }
+ } catch (SQLException e) {
+ ackStream.onError(e);
+ return;
+ }
+ ackStream.onCompleted();
+ };
+ }
+
+ /**
+ * Method responsible to set the parameters, to the preparedStatement object, sent via doPut request.
+ *
+ * @param preparedStatement the preparedStatement object for the operation.
+ * @param root a {@link VectorSchemaRoot} object contain the values to be used in the
+ * PreparedStatement setters.
+ * @param isUpdate a flag to indicate if is an update or query operation.
+ * @throws SQLException in case of error.
+ */
+ private void setDataPreparedStatement(PreparedStatement preparedStatement, VectorSchemaRoot root,
+ boolean isUpdate)
+ throws SQLException {
+ for (int i = 0; i < root.getRowCount(); i++) {
+ for (FieldVector vector : root.getFieldVectors()) {
+ final int vectorPosition = root.getFieldVectors().indexOf(vector);
+ final int position = vectorPosition + 1;
+
+ if (vector instanceof UInt1Vector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (UInt1Vector) vector);
+ } else if (vector instanceof TimeStampNanoTZVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (TimeStampNanoTZVector) vector);
+ } else if (vector instanceof TimeStampMicroTZVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (TimeStampMicroTZVector) vector);
+ } else if (vector instanceof TimeStampMilliTZVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (TimeStampMilliTZVector) vector);
+ } else if (vector instanceof TimeStampSecTZVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (TimeStampSecTZVector) vector);
+ } else if (vector instanceof UInt2Vector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (UInt2Vector) vector);
+ } else if (vector instanceof UInt4Vector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (UInt4Vector) vector);
+ } else if (vector instanceof UInt8Vector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (UInt8Vector) vector);
+ } else if (vector instanceof TinyIntVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (TinyIntVector) vector);
+ } else if (vector instanceof SmallIntVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (SmallIntVector) vector);
+ } else if (vector instanceof IntVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (IntVector) vector);
+ } else if (vector instanceof BigIntVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (BigIntVector) vector);
+ } else if (vector instanceof Float4Vector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (Float4Vector) vector);
+ } else if (vector instanceof Float8Vector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (Float8Vector) vector);
+ } else if (vector instanceof BitVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (BitVector) vector);
+ } else if (vector instanceof DecimalVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (DecimalVector) vector);
+ } else if (vector instanceof Decimal256Vector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (Decimal256Vector) vector);
+ } else if (vector instanceof TimeStampVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (TimeStampVector) vector);
+ } else if (vector instanceof TimeNanoVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (TimeNanoVector) vector);
+ } else if (vector instanceof TimeMicroVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (TimeMicroVector) vector);
+ } else if (vector instanceof TimeMilliVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (TimeMilliVector) vector);
+ } else if (vector instanceof TimeSecVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (TimeSecVector) vector);
+ } else if (vector instanceof DateDayVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (DateDayVector) vector);
+ } else if (vector instanceof DateMilliVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (DateMilliVector) vector);
+ } else if (vector instanceof VarCharVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (VarCharVector) vector);
+ } else if (vector instanceof LargeVarCharVector) {
+ setOnPreparedStatement(preparedStatement, position, vectorPosition, (LargeVarCharVector) vector);
+ }
+ }
+ if (isUpdate) {
+ preparedStatement.addBatch();
+ }
+ }
+ }
+
+ protected TimeZone getTimeZoneForVector(TimeStampVector vector) {
+ ArrowType.Timestamp arrowType = (ArrowType.Timestamp) vector.getField().getFieldType().getType();
+
+ String timezoneName = arrowType.getTimezone();
+ if (timezoneName == null) {
+ return TimeZone.getDefault();
+ }
+
+ return TimeZone.getTimeZone(timezoneName);
+ }
+
+ /**
+ * Set a string parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, VarCharVector vector)
+ throws SQLException {
+ final Text object = vector.getObject(vectorIndex);
+ statement.setObject(column, object.toString());
+ }
+
+ /**
+ * Set a string parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex,
+ LargeVarCharVector vector)
+ throws SQLException {
+ final Text object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set a byte parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, TinyIntVector vector)
+ throws SQLException {
+ final Byte object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set a short parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, SmallIntVector vector)
+ throws SQLException {
+ final Short object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set an integer parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, IntVector vector)
+ throws SQLException {
+ final Integer object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set a long parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, BigIntVector vector)
+ throws SQLException {
+ final Long object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set a float parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, Float4Vector vector)
+ throws SQLException {
+ final Float object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set a double parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, Float8Vector vector)
+ throws SQLException {
+ final Double object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set a BigDecimal parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, DecimalVector vector)
+ throws SQLException {
+ final BigDecimal object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set a BigDecimal parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, Decimal256Vector vector)
+ throws SQLException {
+ final BigDecimal object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set a timestamp parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, TimeStampVector vector)
+ throws SQLException {
+ final Object object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set a time parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, TimeNanoVector vector)
+ throws SQLException {
+ final Long object = vector.getObject(vectorIndex);
+ statement.setTime(column, new Time(object * 1000L));
+ }
+
+ /**
+ * Set a time parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, TimeMicroVector vector)
+ throws SQLException {
+ final Long object = vector.getObject(vectorIndex);
+ statement.setTime(column, new Time(object / 1000L));
+ }
+
+ /**
+ * Set a time parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, TimeMilliVector vector)
+ throws SQLException {
+ final LocalDateTime object = vector.getObject(vectorIndex);
+ statement.setTime(column, Time.valueOf(object.toLocalTime()));
+ }
+
+ /**
+ * Set a time parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, TimeSecVector vector)
+ throws SQLException {
+ final Integer object = vector.getObject(vectorIndex);
+ statement.setTime(column, new Time(object));
+ }
+
+ /**
+ * Set a date parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, DateDayVector vector)
+ throws SQLException {
+ final Integer object = vector.getObject(vectorIndex);
+ statement.setDate(column, new Date(TimeUnit.DAYS.toMillis(object)));
+ }
+
+ /**
+ * Set a date parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, DateMilliVector vector)
+ throws SQLException {
+ final LocalDateTime object = vector.getObject(vectorIndex);
+ statement.setDate(column, Date.valueOf(object.toLocalDate()));
+
+ }
+
+ /**
+ * Set an unsigned 1 byte number parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, UInt1Vector vector)
+ throws SQLException {
+ final Byte object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set an unsigned 2 bytes number parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, UInt2Vector vector)
+ throws SQLException {
+ final Character object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set an unsigned 4 bytes number parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, UInt4Vector vector)
+ throws SQLException {
+ final Integer object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set an unsigned 8 bytes number parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, UInt8Vector vector)
+ throws SQLException {
+ final Long object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set a boolean parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex, BitVector vector)
+ throws SQLException {
+ final Boolean object = vector.getObject(vectorIndex);
+ statement.setObject(column, object);
+ }
+
+ /**
+ * Set a timestamp parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex,
+ TimeStampNanoTZVector vector)
+ throws SQLException {
+ final Long object = vector.getObject(vectorIndex);
+ statement.setTimestamp(column, new Timestamp(object / 1000000L),
+ Calendar.getInstance(getTimeZoneForVector(vector)));
+ }
+
+ /**
+ * Set a timestamp parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex,
+ TimeStampMicroTZVector vector)
+ throws SQLException {
+ final Long object = vector.getObject(vectorIndex);
+ statement.setTimestamp(column, new Timestamp(object / 1000L),
+ Calendar.getInstance(getTimeZoneForVector(vector)));
+ }
+
+ /**
+ * Set a timestamp parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex,
+ TimeStampMilliTZVector vector)
+ throws SQLException {
+ final Long object = vector.getObject(vectorIndex);
+ statement.setTimestamp(column, new Timestamp(object),
+ Calendar.getInstance(getTimeZoneForVector(vector)));
+ }
+
+ /**
+ * Set a timestamp parameter to the preparedStatement object.
+ *
+ * @param statement an instance of the {@link PreparedStatement} class.
+ * @param column the index of the column in the {@link PreparedStatement}.
+ * @param vectorIndex the index from the vector which contain the value.
+ * @param vector an instance of the vector the will be accessed.
+ * @throws SQLException in case of error.
+ */
+ public void setOnPreparedStatement(PreparedStatement statement, int column, int vectorIndex,
+ TimeStampSecTZVector vector)
+ throws SQLException {
+ final Long object = vector.getObject(vectorIndex);
+ statement.setTimestamp(column, new Timestamp(object * 1000L),
+ Calendar.getInstance(getTimeZoneForVector(vector)));
+ }
+
+ @Override
+ public Runnable acceptPutPreparedStatementQuery(CommandPreparedStatementQuery command, CallContext context,
+ FlightStream flightStream, StreamListener ackStream) {
+ final StatementContext statementContext =
+ preparedStatementLoadingCache.getIfPresent(command.getPreparedStatementHandle());
+
+ return () -> {
+ assert statementContext != null;
+ PreparedStatement preparedStatement = statementContext.getStatement();
+
+ try {
+ while (flightStream.next()) {
+ final VectorSchemaRoot root = flightStream.getRoot();
+ setDataPreparedStatement(preparedStatement, root, false);
+ }
+
+ } catch (SQLException e) {
+ ackStream.onError(e);
+ return;
+ }
+ ackStream.onCompleted();
+ };
+ }
+
+ @Override
+ public FlightInfo getFlightInfoSqlInfo(final CommandGetSqlInfo request, final CallContext context,
+ final FlightDescriptor descriptor) {
+ return getFlightInfoForSchema(request, descriptor, Schemas.GET_SQL_INFO_SCHEMA);
+ }
+
+ @Override
+ public void getStreamSqlInfo(final CommandGetSqlInfo command, final CallContext context,
+ final ServerStreamListener listener) {
+ this.sqlInfoBuilder.send(command.getInfoList(), listener);
+ }
+
+ @Override
+ public FlightInfo getFlightInfoCatalogs(final CommandGetCatalogs request, final CallContext context,
+ final FlightDescriptor descriptor) {
+ return getFlightInfoForSchema(request, descriptor, Schemas.GET_CATALOGS_SCHEMA);
+ }
+
+ @Override
+ public void getStreamCatalogs(final CallContext context, final ServerStreamListener listener) {
+ try (final Connection connection = dataSource.getConnection();
+ final ResultSet catalogs = connection.getMetaData().getCatalogs();
+ final VectorSchemaRoot vectorSchemaRoot = getCatalogsRoot(catalogs, rootAllocator)) {
+ listener.start(vectorSchemaRoot);
+ listener.putNext();
+ } catch (SQLException e) {
+ LOGGER.error(format("Failed to getStreamCatalogs: <%s>.", e.getMessage()), e);
+ listener.error(e);
+ } finally {
+ listener.completed();
+ }
+ }
+
+ @Override
+ public FlightInfo getFlightInfoSchemas(final CommandGetDbSchemas request, final CallContext context,
+ final FlightDescriptor descriptor) {
+ return getFlightInfoForSchema(request, descriptor, Schemas.GET_SCHEMAS_SCHEMA);
+ }
+
+ @Override
+ public void getStreamSchemas(final CommandGetDbSchemas command, final CallContext context,
+ final ServerStreamListener listener) {
+ final String catalog = command.hasCatalog() ? command.getCatalog() : null;
+ final String schemaFilterPattern = command.hasDbSchemaFilterPattern() ? command.getDbSchemaFilterPattern() : null;
+ try (final Connection connection = dataSource.getConnection();
+ final ResultSet schemas = connection.getMetaData().getSchemas(catalog, schemaFilterPattern);
+ final VectorSchemaRoot vectorSchemaRoot = getSchemasRoot(schemas, rootAllocator)) {
+ listener.start(vectorSchemaRoot);
+ listener.putNext();
+ } catch (SQLException e) {
+ LOGGER.error(format("Failed to getStreamSchemas: <%s>.", e.getMessage()), e);
+ listener.error(e);
+ } finally {
+ listener.completed();
+ }
+ }
+
+ @Override
+ public FlightInfo getFlightInfoTables(final CommandGetTables request, final CallContext context,
+ final FlightDescriptor descriptor) {
+ return getFlightInfoForSchema(request, descriptor, Schemas.GET_TABLES_SCHEMA);
+ }
+
+ @Override
+ public void getStreamTables(final CommandGetTables command, final CallContext context,
+ final ServerStreamListener listener) {
+ final String catalog = command.hasCatalog() ? command.getCatalog() : null;
+ final String schemaFilterPattern =
+ command.hasDbSchemaFilterPattern() ? command.getDbSchemaFilterPattern() : null;
+ final String tableFilterPattern =
+ command.hasTableNameFilterPattern() ? command.getTableNameFilterPattern() : null;
+
+ final ProtocolStringList protocolStringList = command.getTableTypesList();
+ final int protocolSize = protocolStringList.size();
+ final String[] tableTypes =
+ protocolSize == 0 ? null : protocolStringList.toArray(new String[protocolSize]);
+
+ try (final Connection connection = DriverManager.getConnection(DATABASE_URI);
+ final VectorSchemaRoot vectorSchemaRoot = getTablesRoot(
+ connection.getMetaData(),
+ rootAllocator,
+ command.getIncludeSchema(),
+ catalog, schemaFilterPattern, tableFilterPattern, tableTypes)) {
+ listener.start(vectorSchemaRoot);
+ listener.putNext();
+ } catch (SQLException | IOException e) {
+ LOGGER.error(format("Failed to getStreamTables: <%s>.", e.getMessage()), e);
+ listener.error(e);
+ } finally {
+ listener.completed();
+ }
+ }
+
+ @Override
+ public FlightInfo getFlightInfoTableTypes(final CommandGetTableTypes request, final CallContext context,
+ final FlightDescriptor descriptor) {
+ return getFlightInfoForSchema(request, descriptor, Schemas.GET_TABLE_TYPES_SCHEMA);
+ }
+
+ @Override
+ public void getStreamTableTypes(final CallContext context, final ServerStreamListener listener) {
+ try (final Connection connection = dataSource.getConnection();
+ final ResultSet tableTypes = connection.getMetaData().getTableTypes();
+ final VectorSchemaRoot vectorSchemaRoot = getTableTypesRoot(tableTypes, rootAllocator)) {
+ listener.start(vectorSchemaRoot);
+ listener.putNext();
+ } catch (SQLException e) {
+ LOGGER.error(format("Failed to getStreamTableTypes: <%s>.", e.getMessage()), e);
+ listener.error(e);
+ } finally {
+ listener.completed();
+ }
+ }
+
+ @Override
+ public FlightInfo getFlightInfoPrimaryKeys(final CommandGetPrimaryKeys request, final CallContext context,
+ final FlightDescriptor descriptor) {
+ return getFlightInfoForSchema(request, descriptor, Schemas.GET_PRIMARY_KEYS_SCHEMA);
+ }
+
+ @Override
+ public void getStreamPrimaryKeys(final CommandGetPrimaryKeys command, final CallContext context,
+ final ServerStreamListener listener) {
+
+ final String catalog = command.hasCatalog() ? command.getCatalog() : null;
+ final String schema = command.hasDbSchema() ? command.getDbSchema() : null;
+ final String table = command.getTable();
+
+ try (Connection connection = DriverManager.getConnection(DATABASE_URI)) {
+ final ResultSet primaryKeys = connection.getMetaData().getPrimaryKeys(catalog, schema, table);
+
+ final VarCharVector catalogNameVector = new VarCharVector("catalog_name", rootAllocator);
+ final VarCharVector schemaNameVector = new VarCharVector("schema_name", rootAllocator);
+ final VarCharVector tableNameVector = new VarCharVector("table_name", rootAllocator);
+ final VarCharVector columnNameVector = new VarCharVector("column_name", rootAllocator);
+ final IntVector keySequenceVector = new IntVector("key_sequence", rootAllocator);
+ final VarCharVector keyNameVector = new VarCharVector("key_name", rootAllocator);
+
+ final List vectors =
+ new ArrayList<>(
+ ImmutableList.of(
+ catalogNameVector, schemaNameVector, tableNameVector, columnNameVector, keySequenceVector,
+ keyNameVector));
+ vectors.forEach(FieldVector::allocateNew);
+
+ int rows = 0;
+ for (; primaryKeys.next(); rows++) {
+ saveToVector(primaryKeys.getString("TABLE_CAT"), catalogNameVector, rows);
+ saveToVector(primaryKeys.getString("TABLE_SCHEM"), schemaNameVector, rows);
+ saveToVector(primaryKeys.getString("TABLE_NAME"), tableNameVector, rows);
+ saveToVector(primaryKeys.getString("COLUMN_NAME"), columnNameVector, rows);
+ final int key_seq = primaryKeys.getInt("KEY_SEQ");
+ saveToVector(primaryKeys.wasNull() ? null : key_seq, keySequenceVector, rows);
+ saveToVector(primaryKeys.getString("PK_NAME"), keyNameVector, rows);
+ }
+
+ try (final VectorSchemaRoot vectorSchemaRoot = new VectorSchemaRoot(vectors)) {
+ vectorSchemaRoot.setRowCount(rows);
+
+ listener.start(vectorSchemaRoot);
+ listener.putNext();
+ }
+ } catch (SQLException e) {
+ listener.error(e);
+ } finally {
+ listener.completed();
+ }
+ }
+
+ @Override
+ public FlightInfo getFlightInfoExportedKeys(final CommandGetExportedKeys request, final CallContext context,
+ final FlightDescriptor descriptor) {
+ return getFlightInfoForSchema(request, descriptor, Schemas.GET_EXPORTED_KEYS_SCHEMA);
+ }
+
+ @Override
+ public void getStreamExportedKeys(final CommandGetExportedKeys command, final CallContext context,
+ final ServerStreamListener listener) {
+ String catalog = command.hasCatalog() ? command.getCatalog() : null;
+ String schema = command.hasDbSchema() ? command.getDbSchema() : null;
+ String table = command.getTable();
+
+ try (Connection connection = DriverManager.getConnection(DATABASE_URI);
+ ResultSet keys = connection.getMetaData().getExportedKeys(catalog, schema, table);
+ VectorSchemaRoot vectorSchemaRoot = createVectors(keys)) {
+ listener.start(vectorSchemaRoot);
+ listener.putNext();
+ } catch (SQLException e) {
+ listener.error(e);
+ } finally {
+ listener.completed();
+ }
+ }
+
+ @Override
+ public FlightInfo getFlightInfoImportedKeys(final CommandGetImportedKeys request, final CallContext context,
+ final FlightDescriptor descriptor) {
+ return getFlightInfoForSchema(request, descriptor, Schemas.GET_IMPORTED_KEYS_SCHEMA);
+ }
+
+ @Override
+ public void getStreamImportedKeys(final CommandGetImportedKeys command, final CallContext context,
+ final ServerStreamListener listener) {
+ String catalog = command.hasCatalog() ? command.getCatalog() : null;
+ String schema = command.hasDbSchema() ? command.getDbSchema() : null;
+ String table = command.getTable();
+
+ try (Connection connection = DriverManager.getConnection(DATABASE_URI);
+ ResultSet keys = connection.getMetaData().getImportedKeys(catalog, schema, table);
+ VectorSchemaRoot vectorSchemaRoot = createVectors(keys)) {
+ listener.start(vectorSchemaRoot);
+ listener.putNext();
+ } catch (final SQLException e) {
+ listener.error(e);
+ } finally {
+ listener.completed();
+ }
+ }
+
+ @Override
+ public FlightInfo getFlightInfoCrossReference(CommandGetCrossReference request, CallContext context,
+ FlightDescriptor descriptor) {
+ return getFlightInfoForSchema(request, descriptor, Schemas.GET_CROSS_REFERENCE_SCHEMA);
+ }
+
+ @Override
+ public void getStreamCrossReference(CommandGetCrossReference command, CallContext context,
+ ServerStreamListener listener) {
+ final String pkCatalog = command.hasPkCatalog() ? command.getPkCatalog() : null;
+ final String pkSchema = command.hasPkDbSchema() ? command.getPkDbSchema() : null;
+ final String fkCatalog = command.hasFkCatalog() ? command.getFkCatalog() : null;
+ final String fkSchema = command.hasFkDbSchema() ? command.getFkDbSchema() : null;
+ final String pkTable = command.getPkTable();
+ final String fkTable = command.getFkTable();
+
+ try (Connection connection = DriverManager.getConnection(DATABASE_URI);
+ ResultSet keys = connection.getMetaData()
+ .getCrossReference(pkCatalog, pkSchema, pkTable, fkCatalog, fkSchema, fkTable);
+ VectorSchemaRoot vectorSchemaRoot = createVectors(keys)) {
+ listener.start(vectorSchemaRoot);
+ listener.putNext();
+ } catch (final SQLException e) {
+ listener.error(e);
+ } finally {
+ listener.completed();
+ }
+ }
+
+ private VectorSchemaRoot createVectors(ResultSet keys) throws SQLException {
+ final VarCharVector pkCatalogNameVector = new VarCharVector("pk_catalog_name", rootAllocator);
+ final VarCharVector pkSchemaNameVector = new VarCharVector("pk_schema_name", rootAllocator);
+ final VarCharVector pkTableNameVector = new VarCharVector("pk_table_name", rootAllocator);
+ final VarCharVector pkColumnNameVector = new VarCharVector("pk_column_name", rootAllocator);
+ final VarCharVector fkCatalogNameVector = new VarCharVector("fk_catalog_name", rootAllocator);
+ final VarCharVector fkSchemaNameVector = new VarCharVector("fk_schema_name", rootAllocator);
+ final VarCharVector fkTableNameVector = new VarCharVector("fk_table_name", rootAllocator);
+ final VarCharVector fkColumnNameVector = new VarCharVector("fk_column_name", rootAllocator);
+ final IntVector keySequenceVector = new IntVector("key_sequence", rootAllocator);
+ final VarCharVector fkKeyNameVector = new VarCharVector("fk_key_name", rootAllocator);
+ final VarCharVector pkKeyNameVector = new VarCharVector("pk_key_name", rootAllocator);
+ final UInt1Vector updateRuleVector = new UInt1Vector("update_rule", rootAllocator);
+ final UInt1Vector deleteRuleVector = new UInt1Vector("delete_rule", rootAllocator);
+
+ Map vectorToColumnName = new HashMap<>();
+ vectorToColumnName.put(pkCatalogNameVector, "PKTABLE_CAT");
+ vectorToColumnName.put(pkSchemaNameVector, "PKTABLE_SCHEM");
+ vectorToColumnName.put(pkTableNameVector, "PKTABLE_NAME");
+ vectorToColumnName.put(pkColumnNameVector, "PKCOLUMN_NAME");
+ vectorToColumnName.put(fkCatalogNameVector, "FKTABLE_CAT");
+ vectorToColumnName.put(fkSchemaNameVector, "FKTABLE_SCHEM");
+ vectorToColumnName.put(fkTableNameVector, "FKTABLE_NAME");
+ vectorToColumnName.put(fkColumnNameVector, "FKCOLUMN_NAME");
+ vectorToColumnName.put(keySequenceVector, "KEY_SEQ");
+ vectorToColumnName.put(updateRuleVector, "UPDATE_RULE");
+ vectorToColumnName.put(deleteRuleVector, "DELETE_RULE");
+ vectorToColumnName.put(fkKeyNameVector, "FK_NAME");
+ vectorToColumnName.put(pkKeyNameVector, "PK_NAME");
+
+ final VectorSchemaRoot vectorSchemaRoot = VectorSchemaRoot.of(
+ pkCatalogNameVector, pkSchemaNameVector, pkTableNameVector, pkColumnNameVector, fkCatalogNameVector,
+ fkSchemaNameVector, fkTableNameVector, fkColumnNameVector, keySequenceVector, fkKeyNameVector,
+ pkKeyNameVector, updateRuleVector, deleteRuleVector);
+
+ vectorSchemaRoot.allocateNew();
+ final int rowCount = saveToVectors(vectorToColumnName, keys, true);
+
+ vectorSchemaRoot.setRowCount(rowCount);
+
+ return vectorSchemaRoot;
+ }
+
+ @Override
+ public void getStreamStatement(final TicketStatementQuery ticketStatementQuery, final CallContext context,
+ final ServerStreamListener listener) {
+ final ByteString handle = ticketStatementQuery.getStatementHandle();
+ final StatementContext statementContext =
+ Objects.requireNonNull(statementLoadingCache.getIfPresent(handle));
+ try (final ResultSet resultSet = statementContext.getStatement().getResultSet()) {
+ final Schema schema = jdbcToArrowSchema(resultSet.getMetaData(), DEFAULT_CALENDAR);
+ try (VectorSchemaRoot vectorSchemaRoot = VectorSchemaRoot.create(schema, rootAllocator)) {
+ final VectorLoader loader = new VectorLoader(vectorSchemaRoot);
+ listener.start(vectorSchemaRoot);
+
+ final ArrowVectorIterator iterator = sqlToArrowVectorIterator(resultSet, rootAllocator);
+ while (iterator.hasNext()) {
+ final VectorUnloader unloader = new VectorUnloader(iterator.next());
+ loader.load(unloader.getRecordBatch());
+ listener.putNext();
+ vectorSchemaRoot.clear();
+ }
+
+ listener.putNext();
+ }
+ } catch (SQLException | IOException e) {
+ LOGGER.error(format("Failed to getStreamPreparedStatement: <%s>.", e.getMessage()), e);
+ listener.error(e);
+ } finally {
+ listener.completed();
+ statementLoadingCache.invalidate(handle);
+ }
+ }
+
+ private FlightInfo getFlightInfoForSchema(final T request, final FlightDescriptor descriptor,
+ final Schema schema) {
+ final Ticket ticket = new Ticket(pack(request).toByteArray());
+ // TODO Support multiple endpoints.
+ final List endpoints = singletonList(new FlightEndpoint(ticket, location));
+
+ return new FlightInfo(schema, descriptor, endpoints, -1, -1);
+ }
+
+ private static class StatementRemovalListener
+ implements RemovalListener> {
+ @Override
+ public void onRemoval(final RemovalNotification> notification) {
+ try {
+ AutoCloseables.close(notification.getValue());
+ } catch (final Exception e) {
+ // swallow
+ }
+ }
+ }
+}
diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/StatementContext.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/StatementContext.java
new file mode 100644
index 0000000000000..764ef3f54aae3
--- /dev/null
+++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/StatementContext.java
@@ -0,0 +1,82 @@
+/*
+ * 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 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 the {@link Statement} to be persisted.
+ */
+public final class StatementContext 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;
+ }
+
+ /**
+ * 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 String getQuery() {
+ return 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 statement.equals(that.statement);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(statement);
+ }
+}
diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/util/AdhocTestOption.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/util/AdhocTestOption.java
new file mode 100644
index 0000000000000..6988a86049dbb
--- /dev/null
+++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/util/AdhocTestOption.java
@@ -0,0 +1,45 @@
+/*
+ * 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.util;
+
+import com.google.protobuf.Descriptors.EnumDescriptor;
+import com.google.protobuf.Descriptors.EnumValueDescriptor;
+import com.google.protobuf.ProtocolMessageEnum;
+
+enum AdhocTestOption implements ProtocolMessageEnum {
+ OPTION_A, OPTION_B, OPTION_C;
+
+ @Override
+ public int getNumber() {
+ return ordinal();
+ }
+
+ @Override
+ public EnumValueDescriptor getValueDescriptor() {
+ throw getUnsupportedException();
+ }
+
+ @Override
+ public EnumDescriptor getDescriptorForType() {
+ throw getUnsupportedException();
+ }
+
+ private UnsupportedOperationException getUnsupportedException() {
+ return new UnsupportedOperationException("Unimplemented method is irrelevant for the scope of this test.");
+ }
+}
diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/util/SqlInfoOptionsUtilsBitmaskCreationTest.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/util/SqlInfoOptionsUtilsBitmaskCreationTest.java
new file mode 100644
index 0000000000000..6f2b66646bb20
--- /dev/null
+++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/util/SqlInfoOptionsUtilsBitmaskCreationTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.util;
+
+import static java.util.Arrays.asList;
+import static org.apache.arrow.flight.sql.util.AdhocTestOption.OPTION_A;
+import static org.apache.arrow.flight.sql.util.AdhocTestOption.OPTION_B;
+import static org.apache.arrow.flight.sql.util.AdhocTestOption.OPTION_C;
+import static org.apache.arrow.flight.sql.util.SqlInfoOptionsUtils.createBitmaskFromEnums;
+import static org.hamcrest.CoreMatchers.is;
+
+import java.util.List;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ErrorCollector;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public final class SqlInfoOptionsUtilsBitmaskCreationTest {
+
+ @Parameter
+ public AdhocTestOption[] adhocTestOptions;
+ @Parameter(value = 1)
+ public long expectedBitmask;
+ @Rule
+ public final ErrorCollector collector = new ErrorCollector();
+
+ @Parameters
+ public static List