diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 23f63761c3b5..9b447a2f4e46 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -132,6 +132,7 @@ jobs: # Switch to arrow-flight cd ../arrow-flight + cargo test --features=flight-sql-experimental cargo check --no-default-features # test the --features "simd" of the arrow crate. This requires nightly. diff --git a/arrow-flight/Cargo.toml b/arrow-flight/Cargo.toml index edb4090fb8c4..1d2c2d2ed646 100644 --- a/arrow-flight/Cargo.toml +++ b/arrow-flight/Cargo.toml @@ -32,14 +32,23 @@ base64 = "0.13" tonic = "0.6" bytes = "1" prost = "0.9" +prost-types = { version = "0.9.0", optional = true } prost-derive = "0.9" tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread"] } +futures = { version = "0.3", default-features = false, features = ["alloc"]} + +[features] +default = [] +flight-sql-experimental = ["prost-types"] [dev-dependencies] -futures = { version = "0.3", default-features = false, features = ["alloc"]} [build-dependencies] tonic-build = "0.6" # Pin specific version of the tonic-build dependencies to avoid auto-generated # (and checked in) arrow.flight.protocol.rs from changing proc-macro2 = ">1.0.30" + +[[example]] +name = "flight_sql_server" +required-features = ["flight-sql-experimental"] diff --git a/arrow-flight/build.rs b/arrow-flight/build.rs index 1cbfceb9262c..3df4efad8283 100644 --- a/arrow-flight/build.rs +++ b/arrow-flight/build.rs @@ -49,6 +49,31 @@ fn main() -> Result<(), Box> { file.write_all(buffer.as_bytes())?; } + // override the build location, in order to check in the changes to proto files + env::set_var("OUT_DIR", "src/sql"); + // The current working directory can vary depending on how the project is being + // built or released so we build an absolute path to the proto file + let path = Path::new("../format/FlightSql.proto"); + if path.exists() { + // avoid rerunning build if the file has not changed + println!("cargo:rerun-if-changed=../format/FlightSql.proto"); + + tonic_build::compile_protos("../format/FlightSql.proto")?; + // read file contents to string + let mut file = OpenOptions::new() + .read(true) + .open("src/sql/arrow.flight.protocol.sql.rs")?; + let mut buffer = String::new(); + file.read_to_string(&mut buffer)?; + // append warning that file was auto-generate + let mut file = OpenOptions::new() + .write(true) + .truncate(true) + .open("src/sql/arrow.flight.protocol.sql.rs")?; + file.write_all("// This file was automatically generated through the build.rs script, and should not be edited.\n\n".as_bytes())?; + file.write_all(buffer.as_bytes())?; + } + // As the proto file is checked in, the build should not fail if the file is not found Ok(()) } diff --git a/arrow-flight/examples/flight_sql_server.rs b/arrow-flight/examples/flight_sql_server.rs new file mode 100644 index 000000000000..8b4fe477b868 --- /dev/null +++ b/arrow-flight/examples/flight_sql_server.rs @@ -0,0 +1,240 @@ +// 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. + +use arrow_flight::sql::{ActionCreatePreparedStatementResult, SqlInfo}; +use arrow_flight::FlightData; +use tonic::transport::Server; +use tonic::{Response, Status, Streaming}; + +use arrow_flight::{ + flight_service_server::FlightService, + flight_service_server::FlightServiceServer, + sql::{ + server::FlightSqlService, ActionClosePreparedStatementRequest, + ActionCreatePreparedStatementRequest, CommandGetCatalogs, + CommandGetCrossReference, CommandGetDbSchemas, CommandGetExportedKeys, + CommandGetImportedKeys, CommandGetPrimaryKeys, CommandGetSqlInfo, + CommandGetTableTypes, CommandGetTables, CommandPreparedStatementQuery, + CommandPreparedStatementUpdate, CommandStatementQuery, CommandStatementUpdate, + TicketStatementQuery, + }, + FlightDescriptor, FlightInfo, +}; + +#[derive(Clone)] +pub struct FlightSqlServiceImpl {} + +#[tonic::async_trait] +impl FlightSqlService for FlightSqlServiceImpl { + type FlightService = FlightSqlServiceImpl; + // get_flight_info + async fn get_flight_info_statement( + &self, + _query: CommandStatementQuery, + _request: FlightDescriptor, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn get_flight_info_prepared_statement( + &self, + _query: CommandPreparedStatementQuery, + _request: FlightDescriptor, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn get_flight_info_catalogs( + &self, + _query: CommandGetCatalogs, + _request: FlightDescriptor, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn get_flight_info_schemas( + &self, + _query: CommandGetDbSchemas, + _request: FlightDescriptor, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn get_flight_info_tables( + &self, + _query: CommandGetTables, + _request: FlightDescriptor, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn get_flight_info_table_types( + &self, + _query: CommandGetTableTypes, + _request: FlightDescriptor, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn get_flight_info_sql_info( + &self, + _query: CommandGetSqlInfo, + _request: FlightDescriptor, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn get_flight_info_primary_keys( + &self, + _query: CommandGetPrimaryKeys, + _request: FlightDescriptor, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn get_flight_info_exported_keys( + &self, + _query: CommandGetExportedKeys, + _request: FlightDescriptor, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn get_flight_info_imported_keys( + &self, + _query: CommandGetImportedKeys, + _request: FlightDescriptor, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn get_flight_info_cross_reference( + &self, + _query: CommandGetCrossReference, + _request: FlightDescriptor, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + // do_get + async fn do_get_statement( + &self, + _ticket: TicketStatementQuery, + ) -> Result::DoGetStream>, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + + async fn do_get_prepared_statement( + &self, + _query: CommandPreparedStatementQuery, + ) -> Result::DoGetStream>, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn do_get_catalogs( + &self, + _query: CommandGetCatalogs, + ) -> Result::DoGetStream>, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn do_get_schemas( + &self, + _query: CommandGetDbSchemas, + ) -> Result::DoGetStream>, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn do_get_tables( + &self, + _query: CommandGetTables, + ) -> Result::DoGetStream>, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn do_get_table_types( + &self, + _query: CommandGetTableTypes, + ) -> Result::DoGetStream>, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn do_get_sql_info( + &self, + _query: CommandGetSqlInfo, + ) -> Result::DoGetStream>, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn do_get_primary_keys( + &self, + _query: CommandGetPrimaryKeys, + ) -> Result::DoGetStream>, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn do_get_exported_keys( + &self, + _query: CommandGetExportedKeys, + ) -> Result::DoGetStream>, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn do_get_imported_keys( + &self, + _query: CommandGetImportedKeys, + ) -> Result::DoGetStream>, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn do_get_cross_reference( + &self, + _query: CommandGetCrossReference, + ) -> Result::DoGetStream>, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + // do_put + async fn do_put_statement_update( + &self, + _ticket: CommandStatementUpdate, + ) -> Result { + Err(Status::unimplemented("Not yet implemented")) + } + async fn do_put_prepared_statement_query( + &self, + _query: CommandPreparedStatementQuery, + _request: Streaming, + ) -> Result::DoPutStream>, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + async fn do_put_prepared_statement_update( + &self, + _query: CommandPreparedStatementUpdate, + _request: Streaming, + ) -> Result { + Err(Status::unimplemented("Not yet implemented")) + } + // do_action + async fn do_action_create_prepared_statement( + &self, + _query: ActionCreatePreparedStatementRequest, + ) -> Result { + Err(Status::unimplemented("Not yet implemented")) + } + async fn do_action_close_prepared_statement( + &self, + _query: ActionClosePreparedStatementRequest, + ) { + unimplemented!("Not yet implemented") + } + + async fn register_sql_info(&self, _id: i32, _result: &SqlInfo) {} +} + +/// This example shows how to run a FlightSql server +#[tokio::main] +async fn main() -> Result<(), Box> { + let addr = "0.0.0.0:50051".parse()?; + + let svc = FlightServiceServer::new(FlightSqlServiceImpl {}); + + println!("Listening on {:?}", addr); + + Server::builder().add_service(svc).serve(addr).await?; + + Ok(()) +} diff --git a/arrow-flight/src/lib.rs b/arrow-flight/src/lib.rs index a431cfc08845..5cfbd3f60657 100644 --- a/arrow-flight/src/lib.rs +++ b/arrow-flight/src/lib.rs @@ -67,6 +67,9 @@ pub use gen::Ticket; pub mod utils; +#[cfg(feature = "flight-sql-experimental")] +pub mod sql; + use flight_descriptor::DescriptorType; /// SchemaAsIpc represents a pairing of a `Schema` with IpcWriteOptions diff --git a/arrow-flight/src/sql/arrow.flight.protocol.sql.rs b/arrow-flight/src/sql/arrow.flight.protocol.sql.rs new file mode 100644 index 000000000000..3acb6e248c21 --- /dev/null +++ b/arrow-flight/src/sql/arrow.flight.protocol.sql.rs @@ -0,0 +1,1169 @@ +// This file was automatically generated through the build.rs script, and should not be edited. + +/// +/// 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. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandGetSqlInfo { + /// + /// 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. + #[prost(uint32, repeated, tag = "1")] + pub info: ::prost::alloc::vec::Vec, +} +/// +/// 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. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandGetCatalogs {} +/// +/// 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. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandGetDbSchemas { + /// + /// 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. + #[prost(string, optional, tag = "1")] + pub catalog: ::core::option::Option<::prost::alloc::string::String>, + /// + /// 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. + #[prost(string, optional, tag = "2")] + pub db_schema_filter_pattern: ::core::option::Option<::prost::alloc::string::String>, +} +/// +/// 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. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandGetTables { + /// + /// 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. + #[prost(string, optional, tag = "1")] + pub catalog: ::core::option::Option<::prost::alloc::string::String>, + /// + /// 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. + #[prost(string, optional, tag = "2")] + pub db_schema_filter_pattern: ::core::option::Option<::prost::alloc::string::String>, + /// + /// 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. + #[prost(string, optional, tag = "3")] + pub table_name_filter_pattern: ::core::option::Option<::prost::alloc::string::String>, + /// + /// 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. + #[prost(string, repeated, tag = "4")] + pub table_types: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + /// Specifies if the Arrow schema should be returned for found tables. + #[prost(bool, tag = "5")] + pub include_schema: bool, +} +/// +/// 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. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandGetTableTypes {} +/// +/// 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. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandGetPrimaryKeys { + /// + /// 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. + #[prost(string, optional, tag = "1")] + pub catalog: ::core::option::Option<::prost::alloc::string::String>, + /// + /// 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. + #[prost(string, optional, tag = "2")] + pub db_schema: ::core::option::Option<::prost::alloc::string::String>, + /// Specifies the table to get the primary keys for. + #[prost(string, tag = "3")] + pub table: ::prost::alloc::string::String, +} +/// +/// 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. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandGetExportedKeys { + /// + /// 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. + #[prost(string, optional, tag = "1")] + pub catalog: ::core::option::Option<::prost::alloc::string::String>, + /// + /// 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. + #[prost(string, optional, tag = "2")] + pub db_schema: ::core::option::Option<::prost::alloc::string::String>, + /// Specifies the foreign key table to get the foreign keys for. + #[prost(string, tag = "3")] + pub table: ::prost::alloc::string::String, +} +/// +/// 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 +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandGetImportedKeys { + /// + /// 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. + #[prost(string, optional, tag = "1")] + pub catalog: ::core::option::Option<::prost::alloc::string::String>, + /// + /// 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. + #[prost(string, optional, tag = "2")] + pub db_schema: ::core::option::Option<::prost::alloc::string::String>, + /// Specifies the primary key table to get the foreign keys for. + #[prost(string, tag = "3")] + pub table: ::prost::alloc::string::String, +} +/// +/// 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 +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandGetCrossReference { + ///* + /// 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. + #[prost(string, optional, tag = "1")] + pub pk_catalog: ::core::option::Option<::prost::alloc::string::String>, + ///* + /// 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. + #[prost(string, optional, tag = "2")] + pub pk_db_schema: ::core::option::Option<::prost::alloc::string::String>, + ///* + /// The parent table name. It cannot be null. + #[prost(string, tag = "3")] + pub pk_table: ::prost::alloc::string::String, + ///* + /// 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. + #[prost(string, optional, tag = "4")] + pub fk_catalog: ::core::option::Option<::prost::alloc::string::String>, + ///* + /// 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. + #[prost(string, optional, tag = "5")] + pub fk_db_schema: ::core::option::Option<::prost::alloc::string::String>, + ///* + /// The foreign table name. It cannot be null. + #[prost(string, tag = "6")] + pub fk_table: ::prost::alloc::string::String, +} +// SQL Execution Action Messages + +/// +/// Request message for the "CreatePreparedStatement" action on a Flight SQL enabled backend. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ActionCreatePreparedStatementRequest { + /// The valid SQL string to create a prepared statement for. + #[prost(string, tag = "1")] + pub query: ::prost::alloc::string::String, +} +/// +/// Wrap the result of a "GetPreparedStatement" action. +/// +/// The resultant PreparedStatement can be closed either: +/// - Manually, through the "ClosePreparedStatement" action; +/// - Automatically, by a server timeout. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ActionCreatePreparedStatementResult { + /// Opaque handle for the prepared statement on the server. + #[prost(bytes = "vec", tag = "1")] + pub prepared_statement_handle: ::prost::alloc::vec::Vec, + /// 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. + #[prost(bytes = "vec", tag = "2")] + pub dataset_schema: ::prost::alloc::vec::Vec, + /// 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. + #[prost(bytes = "vec", tag = "3")] + pub parameter_schema: ::prost::alloc::vec::Vec, +} +/// +/// Request message for the "ClosePreparedStatement" action on a Flight SQL enabled backend. +/// Closes server resources associated with the prepared statement handle. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ActionClosePreparedStatementRequest { + /// Opaque handle for the prepared statement on the server. + #[prost(bytes = "vec", tag = "1")] + pub prepared_statement_handle: ::prost::alloc::vec::Vec, +} +// 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. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandStatementQuery { + /// The SQL syntax. + #[prost(string, tag = "1")] + pub query: ::prost::alloc::string::String, +} +///* +/// 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. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TicketStatementQuery { + /// Unique identifier for the instance of the statement to execute. + #[prost(bytes = "vec", tag = "1")] + pub statement_handle: ::prost::alloc::vec::Vec, +} +/// +/// 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. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandPreparedStatementQuery { + /// Opaque handle for the prepared statement on the server. + #[prost(bytes = "vec", tag = "1")] + pub prepared_statement_handle: ::prost::alloc::vec::Vec, +} +/// +/// 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. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandStatementUpdate { + /// The SQL syntax. + #[prost(string, tag = "1")] + pub query: ::prost::alloc::string::String, +} +/// +/// 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. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CommandPreparedStatementUpdate { + /// Opaque handle for the prepared statement on the server. + #[prost(bytes = "vec", tag = "1")] + pub prepared_statement_handle: ::prost::alloc::vec::Vec, +} +/// +/// Returned from the RPC call DoPut when a CommandStatementUpdate +/// CommandPreparedStatementUpdate was in the request, containing +/// results from the update. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DoPutUpdateResult { + /// The number of records updated. A return value of -1 represents + /// an unknown updated record count. + #[prost(int64, tag = "1")] + pub record_count: i64, +} +/// Options for CommandGetSqlInfo. +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub 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. + FlightSqlServerName = 0, + /// Retrieves a UTF-8 string with the native version of the Flight SQL Server. + FlightSqlServerVersion = 1, + /// Retrieves a UTF-8 string with the Arrow format version of the Flight SQL Server. + FlightSqlServerArrowVersion = 2, + /// + /// Retrieves a boolean value indicating whether the Flight SQL Server is read only. + /// + /// Returns: + /// - false: if read-write + /// - true: if read only + FlightSqlServerReadOnly = 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. + SqlDdlCatalog = 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. + SqlDdlSchema = 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. + SqlDdlTable = 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`. + SqlIdentifierCase = 503, + /// Retrieves a UTF-8 string with the supported character(s) used to surround a delimited identifier. + SqlIdentifierQuoteChar = 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`. + SqlQuotedIdentifierCase = 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. + SqlAllTablesAreSelectable = 506, + /// + /// Retrieves the null ordering. + /// + /// Returns a uint32 ordinal for the null ordering being used, as described in + /// `arrow.flight.protocol.sql.SqlNullOrdering`. + SqlNullOrdering = 507, + /// Retrieves a UTF-8 string list with values of the supported keywords. + SqlKeywords = 508, + /// Retrieves a UTF-8 string list with values of the supported numeric functions. + SqlNumericFunctions = 509, + /// Retrieves a UTF-8 string list with values of the supported string functions. + SqlStringFunctions = 510, + /// Retrieves a UTF-8 string list with values of the supported system functions. + SqlSystemFunctions = 511, + /// Retrieves a UTF-8 string list with values of the supported datetime functions. + SqlDatetimeFunctions = 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. + SqlSearchStringEscape = 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 _). + SqlExtraNameCharacters = 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. + SqlSupportsColumnAliasing = 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. + SqlNullPlusNullIsNull = 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> + SqlSupportsConvert = 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. + SqlSupportsTableCorrelationNames = 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 + SqlSupportsDifferentTableCorrelationNames = 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; + SqlSupportsExpressionsInOrderBy = 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. + SqlSupportsOrderByUnrelated = 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`. + SqlSupportedGroupBy = 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. + SqlSupportsLikeEscapeClause = 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. + SqlSupportsNonNullableColumns = 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`. + SqlSupportedGrammar = 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`. + SqlAnsi92SupportedLevel = 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. + SqlSupportsIntegrityEnhancementFacility = 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`. + SqlOuterJoinsSupportLevel = 528, + /// Retrieves a UTF-8 string with the preferred term for "schema". + SqlSchemaTerm = 529, + /// Retrieves a UTF-8 string with the preferred term for "procedure". + SqlProcedureTerm = 530, + /// Retrieves a UTF-8 string with the preferred term for "catalog". + SqlCatalogTerm = 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. + SqlCatalogAtStart = 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`. + SqlSchemasSupportedActions = 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`. + SqlCatalogsSupportedActions = 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`. + SqlSupportedPositionedCommands = 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. + SqlSelectForUpdateSupported = 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. + SqlStoredProceduresSupported = 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`. + SqlSupportedSubqueries = 538, + /// + /// Retrieves a boolean value indicating whether correlated subqueries are supported. + /// + /// Returns: + /// - false: if correlated subqueries are unsupported; + /// - true: if correlated subqueries are supported. + SqlCorrelatedSubqueriesSupported = 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`. + SqlSupportedUnions = 540, + /// Retrieves a uint32 value representing the maximum number of hex characters allowed in an inline binary literal. + SqlMaxBinaryLiteralLength = 541, + /// Retrieves a uint32 value representing the maximum number of characters allowed for a character literal. + SqlMaxCharLiteralLength = 542, + /// Retrieves a uint32 value representing the maximum number of characters allowed for a column name. + SqlMaxColumnNameLength = 543, + /// Retrieves a uint32 value representing the the maximum number of columns allowed in a GROUP BY clause. + SqlMaxColumnsInGroupBy = 544, + /// Retrieves a uint32 value representing the maximum number of columns allowed in an index. + SqlMaxColumnsInIndex = 545, + /// Retrieves a uint32 value representing the maximum number of columns allowed in an ORDER BY clause. + SqlMaxColumnsInOrderBy = 546, + /// Retrieves a uint32 value representing the maximum number of columns allowed in a SELECT list. + SqlMaxColumnsInSelect = 547, + /// Retrieves a uint32 value representing the maximum number of columns allowed in a table. + SqlMaxColumnsInTable = 548, + /// Retrieves a uint32 value representing the maximum number of concurrent connections possible. + SqlMaxConnections = 549, + /// Retrieves a uint32 value the maximum number of characters allowed in a cursor name. + SqlMaxCursorNameLength = 550, + /// + /// Retrieves a uint32 value representing the maximum number of bytes allowed for an index, + /// including all of the parts of the index. + SqlMaxIndexLength = 551, + /// Retrieves a uint32 value representing the maximum number of characters allowed in a schema name. + SqlDbSchemaNameLength = 552, + /// Retrieves a uint32 value representing the maximum number of characters allowed in a procedure name. + SqlMaxProcedureNameLength = 553, + /// Retrieves a uint32 value representing the maximum number of characters allowed in a catalog name. + SqlMaxCatalogNameLength = 554, + /// Retrieves a uint32 value representing the maximum number of bytes allowed in a single row. + SqlMaxRowSize = 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. + SqlMaxRowSizeIncludesBlobs = 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. + SqlMaxStatementLength = 557, + /// Retrieves a uint32 value representing the maximum number of active statements that can be open at the same time. + SqlMaxStatements = 558, + /// Retrieves a uint32 value representing the maximum number of characters allowed in a table name. + SqlMaxTableNameLength = 559, + /// Retrieves a uint32 value representing the maximum number of tables allowed in a SELECT statement. + SqlMaxTablesInSelect = 560, + /// Retrieves a uint32 value representing the maximum number of characters allowed in a user name. + SqlMaxUsernameLength = 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. + SqlDefaultTransactionIsolation = 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. + SqlTransactionsSupported = 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`. + SqlSupportedTransactionsIsolationLevels = 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. + SqlDataDefinitionCausesTransactionCommit = 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. + SqlDataDefinitionsInTransactionsIgnored = 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`. + SqlSupportedResultSetTypes = 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`. + SqlSupportedConcurrenciesForResultSetUnspecified = 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`. + SqlSupportedConcurrenciesForResultSetForwardOnly = 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`. + SqlSupportedConcurrenciesForResultSetScrollSensitive = 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`. + SqlSupportedConcurrenciesForResultSetScrollInsensitive = 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. + SqlBatchUpdatesSupported = 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. + SqlSavepointsSupported = 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. + SqlNamedParametersSupported = 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. + SqlLocatorsUpdateCopy = 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. + SqlStoredFunctionsUsingCallSyntaxSupported = 576, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlSupportedCaseSensitivity { + SqlCaseSensitivityUnknown = 0, + SqlCaseSensitivityCaseInsensitive = 1, + SqlCaseSensitivityUppercase = 2, + SqlCaseSensitivityLowercase = 3, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlNullOrdering { + SqlNullsSortedHigh = 0, + SqlNullsSortedLow = 1, + SqlNullsSortedAtStart = 2, + SqlNullsSortedAtEnd = 3, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SupportedSqlGrammar { + SqlMinimumGrammar = 0, + SqlCoreGrammar = 1, + SqlExtendedGrammar = 2, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SupportedAnsi92SqlGrammarLevel { + Ansi92EntrySql = 0, + Ansi92IntermediateSql = 1, + Ansi92FullSql = 2, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlOuterJoinsSupportLevel { + SqlJoinsUnsupported = 0, + SqlLimitedOuterJoins = 1, + SqlFullOuterJoins = 2, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlSupportedGroupBy { + SqlGroupByUnrelated = 0, + SqlGroupByBeyondSelect = 1, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlSupportedElementActions { + SqlElementInProcedureCalls = 0, + SqlElementInIndexDefinitions = 1, + SqlElementInPrivilegeDefinitions = 2, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlSupportedPositionedCommands { + SqlPositionedDelete = 0, + SqlPositionedUpdate = 1, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlSupportedSubqueries { + SqlSubqueriesInComparisons = 0, + SqlSubqueriesInExists = 1, + SqlSubqueriesInIns = 2, + SqlSubqueriesInQuantifieds = 3, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlSupportedUnions { + SqlUnion = 0, + SqlUnionAll = 1, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlTransactionIsolationLevel { + SqlTransactionNone = 0, + SqlTransactionReadUncommitted = 1, + SqlTransactionReadCommitted = 2, + SqlTransactionRepeatableRead = 3, + SqlTransactionSerializable = 4, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlSupportedTransactions { + SqlTransactionUnspecified = 0, + SqlDataDefinitionTransactions = 1, + SqlDataManipulationTransactions = 2, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlSupportedResultSetType { + SqlResultSetTypeUnspecified = 0, + SqlResultSetTypeForwardOnly = 1, + SqlResultSetTypeScrollInsensitive = 2, + SqlResultSetTypeScrollSensitive = 3, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlSupportedResultSetConcurrency { + SqlResultSetConcurrencyUnspecified = 0, + SqlResultSetConcurrencyReadOnly = 1, + SqlResultSetConcurrencyUpdatable = 2, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum SqlSupportsConvert { + SqlConvertBigint = 0, + SqlConvertBinary = 1, + SqlConvertBit = 2, + SqlConvertChar = 3, + SqlConvertDate = 4, + SqlConvertDecimal = 5, + SqlConvertFloat = 6, + SqlConvertInteger = 7, + SqlConvertIntervalDayTime = 8, + SqlConvertIntervalYearMonth = 9, + SqlConvertLongvarbinary = 10, + SqlConvertLongvarchar = 11, + SqlConvertNumeric = 12, + SqlConvertReal = 13, + SqlConvertSmallint = 14, + SqlConvertTime = 15, + SqlConvertTimestamp = 16, + SqlConvertTinyint = 17, + SqlConvertVarbinary = 18, + SqlConvertVarchar = 19, +} +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] +#[repr(i32)] +pub enum UpdateDeleteRules { + Cascade = 0, + Restrict = 1, + SetNull = 2, + NoAction = 3, + SetDefault = 4, +} diff --git a/arrow-flight/src/sql/mod.rs b/arrow-flight/src/sql/mod.rs new file mode 100644 index 000000000000..cd198a1401d1 --- /dev/null +++ b/arrow-flight/src/sql/mod.rs @@ -0,0 +1,178 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::error::{ArrowError, Result as ArrowResult}; +use prost::Message; + +mod gen { + #![allow(clippy::all)] + include!("arrow.flight.protocol.sql.rs"); +} + +pub use gen::ActionClosePreparedStatementRequest; +pub use gen::ActionCreatePreparedStatementRequest; +pub use gen::ActionCreatePreparedStatementResult; +pub use gen::CommandGetCatalogs; +pub use gen::CommandGetCrossReference; +pub use gen::CommandGetDbSchemas; +pub use gen::CommandGetExportedKeys; +pub use gen::CommandGetImportedKeys; +pub use gen::CommandGetPrimaryKeys; +pub use gen::CommandGetSqlInfo; +pub use gen::CommandGetTableTypes; +pub use gen::CommandGetTables; +pub use gen::CommandPreparedStatementQuery; +pub use gen::CommandPreparedStatementUpdate; +pub use gen::CommandStatementQuery; +pub use gen::CommandStatementUpdate; +pub use gen::DoPutUpdateResult; +pub use gen::SqlInfo; +pub use gen::SqlNullOrdering; +pub use gen::SqlOuterJoinsSupportLevel; +pub use gen::SqlSupportedCaseSensitivity; +pub use gen::SqlSupportedElementActions; +pub use gen::SqlSupportedGroupBy; +pub use gen::SqlSupportedPositionedCommands; +pub use gen::SqlSupportedResultSetConcurrency; +pub use gen::SqlSupportedResultSetType; +pub use gen::SqlSupportedSubqueries; +pub use gen::SqlSupportedTransactions; +pub use gen::SqlSupportedUnions; +pub use gen::SqlSupportsConvert; +pub use gen::SqlTransactionIsolationLevel; +pub use gen::SupportedSqlGrammar; +pub use gen::TicketStatementQuery; +pub use gen::UpdateDeleteRules; + +pub mod server; + +/// ProstMessageExt are useful utility methods for prost::Message types +pub trait ProstMessageExt: prost::Message + Default { + /// type_url for this Message + fn type_url() -> &'static str; + + /// Convert this Message to prost_types::Any + fn as_any(&self) -> prost_types::Any; +} + +macro_rules! prost_message_ext { + ($($name:ty,)*) => { + $( + impl ProstMessageExt for $name { + fn type_url() -> &'static str { + concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name)) + } + + fn as_any(&self) -> prost_types::Any { + prost_types::Any { + type_url: <$name>::type_url().to_string(), + value: self.encode_to_vec(), + } + } + } + )* + }; +} + +// Implement ProstMessageExt for all structs defined in FlightSql.proto +prost_message_ext!( + ActionClosePreparedStatementRequest, + ActionCreatePreparedStatementRequest, + ActionCreatePreparedStatementResult, + CommandGetCatalogs, + CommandGetCrossReference, + CommandGetDbSchemas, + CommandGetExportedKeys, + CommandGetImportedKeys, + CommandGetPrimaryKeys, + CommandGetSqlInfo, + CommandGetTableTypes, + CommandGetTables, + CommandPreparedStatementQuery, + CommandPreparedStatementUpdate, + CommandStatementQuery, + CommandStatementUpdate, + DoPutUpdateResult, + TicketStatementQuery, +); + +/// ProstAnyExt are useful utility methods for prost_types::Any +/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs) +pub trait ProstAnyExt { + /// Check if `Any` contains a message of given type. + fn is(&self) -> bool; + + /// Extract a message from this `Any`. + /// + /// # Returns + /// + /// * `Ok(None)` when message type mismatch + /// * `Err` when parse failed + fn unpack(&self) -> ArrowResult>; + + /// Pack any message into `prost_types::Any` value. + fn pack(message: &M) -> ArrowResult; +} + +impl ProstAnyExt for prost_types::Any { + fn is(&self) -> bool { + M::type_url() == self.type_url + } + + fn unpack(&self) -> ArrowResult> { + if !self.is::() { + return Ok(None); + } + let m = prost::Message::decode(&*self.value).map_err(|err| { + ArrowError::ParseError(format!("Unable to decode Any value: {}", err)) + })?; + Ok(Some(m)) + } + + fn pack(message: &M) -> ArrowResult { + Ok(message.as_any()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_type_url() { + assert_eq!( + TicketStatementQuery::type_url(), + "type.googleapis.com/arrow.flight.protocol.sql.TicketStatementQuery" + ); + assert_eq!( + CommandStatementQuery::type_url(), + "type.googleapis.com/arrow.flight.protocol.sql.CommandStatementQuery" + ); + } + + #[test] + fn test_prost_any_pack_unpack() -> ArrowResult<()> { + let query = CommandStatementQuery { + query: "select 1".to_string(), + }; + let any = prost_types::Any::pack(&query)?; + assert!(any.is::()); + let unpack_query: CommandStatementQuery = any.unpack()?.unwrap(); + assert_eq!(query, unpack_query); + Ok(()) + } +} diff --git a/arrow-flight/src/sql/server.rs b/arrow-flight/src/sql/server.rs new file mode 100644 index 000000000000..87e282b103b7 --- /dev/null +++ b/arrow-flight/src/sql/server.rs @@ -0,0 +1,658 @@ +// 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. + +use std::pin::Pin; + +use futures::Stream; +use prost::Message; +use tonic::{Request, Response, Status, Streaming}; + +use super::{ + super::{ + flight_service_server::FlightService, Action, ActionType, Criteria, Empty, + FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse, + PutResult, SchemaResult, Ticket, + }, + ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest, + ActionCreatePreparedStatementResult, CommandGetCatalogs, CommandGetCrossReference, + CommandGetDbSchemas, CommandGetExportedKeys, CommandGetImportedKeys, + CommandGetPrimaryKeys, CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables, + CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery, + CommandStatementUpdate, DoPutUpdateResult, ProstAnyExt, ProstMessageExt, SqlInfo, + TicketStatementQuery, +}; + +static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement"; +static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement"; + +/// Implements FlightSqlService to handle the flight sql protocol +#[tonic::async_trait] +pub trait FlightSqlService: + std::marker::Sync + std::marker::Send + std::marker::Sized + 'static +{ + /// When impl FlightSqlService, you can always set FlightService to Self + type FlightService: FlightService; + + /// Get a FlightInfo for executing a SQL query. + async fn get_flight_info_statement( + &self, + query: CommandStatementQuery, + request: FlightDescriptor, + ) -> Result, Status>; + + /// Get a FlightInfo for executing an already created prepared statement. + async fn get_flight_info_prepared_statement( + &self, + query: CommandPreparedStatementQuery, + request: FlightDescriptor, + ) -> Result, Status>; + + /// Get a FlightInfo for listing catalogs. + async fn get_flight_info_catalogs( + &self, + query: CommandGetCatalogs, + request: FlightDescriptor, + ) -> Result, Status>; + + /// Get a FlightInfo for listing schemas. + async fn get_flight_info_schemas( + &self, + query: CommandGetDbSchemas, + request: FlightDescriptor, + ) -> Result, Status>; + + /// Get a FlightInfo for listing tables. + async fn get_flight_info_tables( + &self, + query: CommandGetTables, + request: FlightDescriptor, + ) -> Result, Status>; + + /// Get a FlightInfo to extract information about the table types. + async fn get_flight_info_table_types( + &self, + query: CommandGetTableTypes, + request: FlightDescriptor, + ) -> Result, Status>; + + /// Get a FlightInfo for retrieving other information (See SqlInfo). + async fn get_flight_info_sql_info( + &self, + query: CommandGetSqlInfo, + request: FlightDescriptor, + ) -> Result, Status>; + + /// Get a FlightInfo to extract information about primary and foreign keys. + async fn get_flight_info_primary_keys( + &self, + query: CommandGetPrimaryKeys, + request: FlightDescriptor, + ) -> Result, Status>; + + /// Get a FlightInfo to extract information about exported keys. + async fn get_flight_info_exported_keys( + &self, + query: CommandGetExportedKeys, + request: FlightDescriptor, + ) -> Result, Status>; + + /// Get a FlightInfo to extract information about imported keys. + async fn get_flight_info_imported_keys( + &self, + query: CommandGetImportedKeys, + request: FlightDescriptor, + ) -> Result, Status>; + + /// Get a FlightInfo to extract information about cross reference. + async fn get_flight_info_cross_reference( + &self, + query: CommandGetCrossReference, + request: FlightDescriptor, + ) -> Result, Status>; + + // do_get + + /// Get a FlightDataStream containing the query results. + async fn do_get_statement( + &self, + ticket: TicketStatementQuery, + ) -> Result::DoGetStream>, Status>; + + /// Get a FlightDataStream containing the prepared statement query results. + async fn do_get_prepared_statement( + &self, + query: CommandPreparedStatementQuery, + ) -> Result::DoGetStream>, Status>; + + /// Get a FlightDataStream containing the list of catalogs. + async fn do_get_catalogs( + &self, + query: CommandGetCatalogs, + ) -> Result::DoGetStream>, Status>; + + /// Get a FlightDataStream containing the list of schemas. + async fn do_get_schemas( + &self, + query: CommandGetDbSchemas, + ) -> Result::DoGetStream>, Status>; + + /// Get a FlightDataStream containing the list of tables. + async fn do_get_tables( + &self, + query: CommandGetTables, + ) -> Result::DoGetStream>, Status>; + + /// Get a FlightDataStream containing the data related to the table types. + async fn do_get_table_types( + &self, + query: CommandGetTableTypes, + ) -> Result::DoGetStream>, Status>; + + /// Get a FlightDataStream containing the list of SqlInfo results. + async fn do_get_sql_info( + &self, + query: CommandGetSqlInfo, + ) -> Result::DoGetStream>, Status>; + + /// Get a FlightDataStream containing the data related to the primary and foreign keys. + async fn do_get_primary_keys( + &self, + query: CommandGetPrimaryKeys, + ) -> Result::DoGetStream>, Status>; + + /// Get a FlightDataStream containing the data related to the exported keys. + async fn do_get_exported_keys( + &self, + query: CommandGetExportedKeys, + ) -> Result::DoGetStream>, Status>; + + /// Get a FlightDataStream containing the data related to the imported keys. + async fn do_get_imported_keys( + &self, + query: CommandGetImportedKeys, + ) -> Result::DoGetStream>, Status>; + + /// Get a FlightDataStream containing the data related to the cross reference. + async fn do_get_cross_reference( + &self, + query: CommandGetCrossReference, + ) -> Result::DoGetStream>, Status>; + + // do_put + + /// Execute an update SQL statement. + async fn do_put_statement_update( + &self, + ticket: CommandStatementUpdate, + ) -> Result; + + /// Bind parameters to given prepared statement. + async fn do_put_prepared_statement_query( + &self, + query: CommandPreparedStatementQuery, + request: Streaming, + ) -> Result::DoPutStream>, Status>; + + /// Execute an update SQL prepared statement. + async fn do_put_prepared_statement_update( + &self, + query: CommandPreparedStatementUpdate, + request: Streaming, + ) -> Result; + + // do_action + + /// Create a prepared statement from given SQL statement. + async fn do_action_create_prepared_statement( + &self, + query: ActionCreatePreparedStatementRequest, + ) -> Result; + + /// Close a prepared statement. + async fn do_action_close_prepared_statement( + &self, + query: ActionClosePreparedStatementRequest, + ); + + /// Register a new SqlInfo result, making it available when calling GetSqlInfo. + async fn register_sql_info(&self, id: i32, result: &SqlInfo); +} + +/// Implements the lower level interface to handle FlightSQL +#[tonic::async_trait] +impl FlightService for T +where + T: FlightSqlService + std::marker::Send, +{ + type HandshakeStream = + Pin> + Send + 'static>>; + type ListFlightsStream = + Pin> + Send + 'static>>; + type DoGetStream = + Pin> + Send + 'static>>; + type DoPutStream = + Pin> + Send + 'static>>; + type DoActionStream = Pin< + Box> + Send + 'static>, + >; + type ListActionsStream = + Pin> + Send + 'static>>; + type DoExchangeStream = + Pin> + Send + 'static>>; + + async fn handshake( + &self, + _request: Request>, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + + async fn list_flights( + &self, + _request: Request, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + + async fn get_flight_info( + &self, + request: Request, + ) -> Result, Status> { + let request = request.into_inner(); + let any: prost_types::Any = + prost::Message::decode(&*request.cmd).map_err(decode_error_to_status)?; + + if any.is::() { + return self + .get_flight_info_statement( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await; + } + if any.is::() { + return self + .get_flight_info_prepared_statement( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await; + } + if any.is::() { + return self + .get_flight_info_catalogs( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await; + } + if any.is::() { + return self + .get_flight_info_schemas( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await; + } + if any.is::() { + return self + .get_flight_info_tables( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await; + } + if any.is::() { + return self + .get_flight_info_table_types( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await; + } + if any.is::() { + return self + .get_flight_info_sql_info( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await; + } + if any.is::() { + return self + .get_flight_info_primary_keys( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await; + } + if any.is::() { + return self + .get_flight_info_exported_keys( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await; + } + if any.is::() { + return self + .get_flight_info_imported_keys( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await; + } + if any.is::() { + return self + .get_flight_info_cross_reference( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await; + } + + Err(Status::unimplemented(format!( + "get_flight_info: The defined request is invalid: {:?}", + String::from_utf8(any.encode_to_vec()).unwrap() + ))) + } + + async fn get_schema( + &self, + _request: Request, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } + + async fn do_get( + &self, + request: Request, + ) -> Result, Status> { + let request = request.into_inner(); + let any: prost_types::Any = + prost::Message::decode(&*request.ticket).map_err(decode_error_to_status)?; + + if any.is::() { + return self + .do_get_statement( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + ) + .await; + } + if any.is::() { + return self + .do_get_prepared_statement( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + ) + .await; + } + if any.is::() { + return self + .do_get_catalogs( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + ) + .await; + } + if any.is::() { + return self + .do_get_schemas( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + ) + .await; + } + if any.is::() { + return self + .do_get_tables( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + ) + .await; + } + if any.is::() { + return self + .do_get_table_types( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + ) + .await; + } + if any.is::() { + return self + .do_get_sql_info( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + ) + .await; + } + if any.is::() { + return self + .do_get_primary_keys( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + ) + .await; + } + if any.is::() { + return self + .do_get_exported_keys( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + ) + .await; + } + if any.is::() { + return self + .do_get_imported_keys( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + ) + .await; + } + if any.is::() { + return self + .do_get_cross_reference( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + ) + .await; + } + + Err(Status::unimplemented(format!( + "do_get: The defined request is invalid: {:?}", + String::from_utf8(request.ticket).unwrap() + ))) + } + + async fn do_put( + &self, + request: Request>, + ) -> Result, Status> { + let mut request = request.into_inner(); + let cmd = request.message().await?.unwrap(); + let any: prost_types::Any = + prost::Message::decode(&*cmd.flight_descriptor.unwrap().cmd) + .map_err(decode_error_to_status)?; + if any.is::() { + let record_count = self + .do_put_statement_update( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + ) + .await?; + let result = DoPutUpdateResult { record_count }; + let output = futures::stream::iter(vec![Ok(super::super::gen::PutResult { + app_metadata: result.as_any().encode_to_vec(), + })]); + return Ok(Response::new(Box::pin(output))); + } + if any.is::() { + return self + .do_put_prepared_statement_query( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await; + } + if any.is::() { + let record_count = self + .do_put_prepared_statement_update( + any.unpack() + .map_err(arrow_error_to_status)? + .expect("unreachable"), + request, + ) + .await?; + let result = DoPutUpdateResult { record_count }; + let output = futures::stream::iter(vec![Ok(super::super::gen::PutResult { + app_metadata: result.as_any().encode_to_vec(), + })]); + return Ok(Response::new(Box::pin(output))); + } + + Err(Status::invalid_argument(format!( + "do_put: The defined request is invalid: {:?}", + String::from_utf8(any.encode_to_vec()).unwrap() + ))) + } + + async fn list_actions( + &self, + _request: Request, + ) -> Result, Status> { + let create_prepared_statement_action_type = ActionType { + r#type: CREATE_PREPARED_STATEMENT.to_string(), + description: "Creates a reusable prepared statement resource on the server.\n + Request Message: ActionCreatePreparedStatementRequest\n + Response Message: ActionCreatePreparedStatementResult" + .into(), + }; + let close_prepared_statement_action_type = ActionType { + r#type: CLOSE_PREPARED_STATEMENT.to_string(), + description: "Closes a reusable prepared statement resource on the server.\n + Request Message: ActionClosePreparedStatementRequest\n + Response Message: N/A" + .into(), + }; + let actions: Vec> = vec![ + Ok(create_prepared_statement_action_type), + Ok(close_prepared_statement_action_type), + ]; + let output = futures::stream::iter(actions); + Ok(Response::new(Box::pin(output) as Self::ListActionsStream)) + } + + async fn do_action( + &self, + request: Request, + ) -> Result, Status> { + let request = request.into_inner(); + + if request.r#type == CREATE_PREPARED_STATEMENT { + let any: prost_types::Any = + prost::Message::decode(&*request.body).map_err(decode_error_to_status)?; + + let cmd: ActionCreatePreparedStatementRequest = any + .unpack() + .map_err(arrow_error_to_status)? + .ok_or_else(|| { + Status::invalid_argument( + "Unable to unpack ActionCreatePreparedStatementRequest.", + ) + })?; + let stmt = self.do_action_create_prepared_statement(cmd).await?; + let output = futures::stream::iter(vec![Ok(super::super::gen::Result { + body: stmt.as_any().encode_to_vec(), + })]); + return Ok(Response::new(Box::pin(output))); + } + if request.r#type == CLOSE_PREPARED_STATEMENT { + let any: prost_types::Any = + prost::Message::decode(&*request.body).map_err(decode_error_to_status)?; + + let cmd: ActionClosePreparedStatementRequest = any + .unpack() + .map_err(arrow_error_to_status)? + .ok_or_else(|| { + Status::invalid_argument( + "Unable to unpack ActionClosePreparedStatementRequest.", + ) + })?; + self.do_action_close_prepared_statement(cmd).await; + return Ok(Response::new(Box::pin(futures::stream::empty()))); + } + + Err(Status::invalid_argument(format!( + "do_action: The defined request is invalid: {:?}", + request.r#type + ))) + } + + async fn do_exchange( + &self, + _request: Request>, + ) -> Result, Status> { + Err(Status::unimplemented("Not yet implemented")) + } +} + +fn decode_error_to_status(err: prost::DecodeError) -> tonic::Status { + tonic::Status::invalid_argument(format!("{:?}", err)) +} + +fn arrow_error_to_status(err: arrow::error::ArrowError) -> tonic::Status { + tonic::Status::internal(format!("{:?}", err)) +} diff --git a/dev/release/rat_exclude_files.txt b/dev/release/rat_exclude_files.txt index 92b110879962..c7996a78af86 100644 --- a/dev/release/rat_exclude_files.txt +++ b/dev/release/rat_exclude_files.txt @@ -18,4 +18,5 @@ conbench/.flake8 conbench/.isort.cfg # auto-generated arrow-flight/src/arrow.flight.protocol.rs +arrow-flight/src/sql/arrow.flight.protocol.sql.rs .github/* diff --git a/format/FlightSql.proto b/format/FlightSql.proto new file mode 100644 index 000000000000..3e85e348bc9c --- /dev/null +++ b/format/FlightSql.proto @@ -0,0 +1,1337 @@ +/* + * 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. + */ + +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; +}