diff --git a/ci/docker/conda-cpp.dockerfile b/ci/docker/conda-cpp.dockerfile index 8fd5e46fd6d..9363e67f796 100644 --- a/ci/docker/conda-cpp.dockerfile +++ b/ci/docker/conda-cpp.dockerfile @@ -41,6 +41,7 @@ ENV ARROW_BUILD_TESTS=ON \ ARROW_DATASET=ON \ ARROW_DEPENDENCY_SOURCE=CONDA \ ARROW_FLIGHT=ON \ + ARROW_FLIGHT_SQL=ON \ ARROW_GANDIVA=ON \ ARROW_HOME=$CONDA_PREFIX \ ARROW_ORC=ON \ diff --git a/ci/scripts/cpp_build.sh b/ci/scripts/cpp_build.sh index f791ddd5645..02718e57836 100755 --- a/ci/scripts/cpp_build.sh +++ b/ci/scripts/cpp_build.sh @@ -70,6 +70,7 @@ cmake \ -DARROW_EXTRA_ERROR_CONTEXT=${ARROW_EXTRA_ERROR_CONTEXT:-OFF} \ -DARROW_FILESYSTEM=${ARROW_FILESYSTEM:-ON} \ -DARROW_FLIGHT=${ARROW_FLIGHT:-OFF} \ + -DARROW_FLIGHT_SQL=${ARROW_FLIGHT_SQL:-OFF} \ -DARROW_FUZZING=${ARROW_FUZZING:-OFF} \ -DARROW_GANDIVA_JAVA=${ARROW_GANDIVA_JAVA:-OFF} \ -DARROW_GANDIVA_PC_CXX_FLAGS=${ARROW_GANDIVA_PC_CXX_FLAGS:-} \ diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 0262357d6c7..e2b9f4e39b9 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -334,6 +334,10 @@ if(ARROW_GANDIVA) set(ARROW_WITH_RE2 ON) endif() +if(ARROW_FLIGHT_SQL) + set(ARROW_FLIGHT ON) +endif() + if(ARROW_CUDA OR ARROW_FLIGHT OR ARROW_PARQUET diff --git a/cpp/cmake_modules/DefineOptions.cmake b/cpp/cmake_modules/DefineOptions.cmake index f2ddff3997d..2afbdab4a40 100644 --- a/cpp/cmake_modules/DefineOptions.cmake +++ b/cpp/cmake_modules/DefineOptions.cmake @@ -226,6 +226,8 @@ if("${CMAKE_SOURCE_DIR}" STREQUAL "${CMAKE_CURRENT_SOURCE_DIR}") define_option(ARROW_FLIGHT "Build the Arrow Flight RPC System (requires GRPC, Protocol Buffers)" OFF) + define_option(ARROW_FLIGHT_SQL "Build the Arrow Flight SQL extension" OFF) + define_option(ARROW_GANDIVA "Build the Gandiva libraries" OFF) define_option(ARROW_GCS diff --git a/cpp/cmake_modules/FindArrowFlightSql.cmake b/cpp/cmake_modules/FindArrowFlightSql.cmake new file mode 100644 index 00000000000..cbca81cac44 --- /dev/null +++ b/cpp/cmake_modules/FindArrowFlightSql.cmake @@ -0,0 +1,93 @@ +# 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. + +# - Find Arrow Flight SQL +# +# This module requires Arrow from which it uses +# arrow_find_package() +# +# This module defines +# ARROW_FLIGHT_SQL_FOUND, whether Flight has been found +# ARROW_FLIGHT_SQL_IMPORT_LIB, +# path to libarrow_flight's import library (Windows only) +# ARROW_FLIGHT_SQL_INCLUDE_DIR, directory containing headers +# ARROW_FLIGHT_SQL_LIBS, deprecated. Use ARROW_FLIGHT_SQL_LIB_DIR instead +# ARROW_FLIGHT_SQL_LIB_DIR, directory containing Flight libraries +# ARROW_FLIGHT_SQL_SHARED_IMP_LIB, deprecated. Use ARROW_FLIGHT_SQL_IMPORT_LIB instead +# ARROW_FLIGHT_SQL_SHARED_LIB, path to libarrow_flight's shared library +# ARROW_FLIGHT_SQL_STATIC_LIB, path to libarrow_flight.a + +if(DEFINED ARROW_FLIGHT_SQL_FOUND) + return() +endif() + +set(find_package_arguments) +if(${CMAKE_FIND_PACKAGE_NAME}_FIND_VERSION) + list(APPEND find_package_arguments "${${CMAKE_FIND_PACKAGE_NAME}_FIND_VERSION}") +endif() +if(${CMAKE_FIND_PACKAGE_NAME}_FIND_REQUIRED) + list(APPEND find_package_arguments REQUIRED) +endif() +if(${CMAKE_FIND_PACKAGE_NAME}_FIND_QUIETLY) + list(APPEND find_package_arguments QUIET) +endif() +find_package(Arrow ${find_package_arguments}) + +if(ARROW_FOUND) + arrow_find_package(ARROW_FLIGHT_SQL + "${ARROW_HOME}" + arrow_flight_sql + arrow/flight/sql/api.h + ArrowFlightSql + arrow-flight-sql) + if(NOT ARROW_FLIGHT_SQL_VERSION) + set(ARROW_FLIGHT_SQL_VERSION "${ARROW_VERSION}") + endif() +endif() + +if("${ARROW_FLIGHT_SQL_VERSION}" VERSION_EQUAL "${ARROW_VERSION}") + set(ARROW_FLIGHT_SQL_VERSION_MATCH TRUE) +else() + set(ARROW_FLIGHT_SQL_VERSION_MATCH FALSE) +endif() + +mark_as_advanced(ARROW_FLIGHT_SQL_IMPORT_LIB + ARROW_FLIGHT_SQL_INCLUDE_DIR + ARROW_FLIGHT_SQL_LIBS + ARROW_FLIGHT_SQL_LIB_DIR + ARROW_FLIGHT_SQL_SHARED_IMP_LIB + ARROW_FLIGHT_SQL_SHARED_LIB + ARROW_FLIGHT_SQL_STATIC_LIB + ARROW_FLIGHT_SQL_VERSION + ARROW_FLIGHT_SQL_VERSION_MATCH) + +find_package_handle_standard_args( + ArrowFlightSql + REQUIRED_VARS ARROW_FLIGHT_SQL_INCLUDE_DIR ARROW_FLIGHT_SQL_LIB_DIR + ARROW_FLIGHT_SQL_VERSION_MATCH + VERSION_VAR ARROW_FLIGHT_SQL_VERSION) +set(ARROW_FLIGHT_SQL_FOUND ${ArrowFlightSql_FOUND}) + +if(ArrowFlightSql_FOUND AND NOT ArrowFlightSql_FIND_QUIETLY) + message(STATUS "Found the Arrow Flight SQL by ${ARROW_FLIGHT_SQL_FIND_APPROACH}") + message(STATUS "Found the Arrow Flight SQL shared library: ${ARROW_FLIGHT_SQL_SHARED_LIB}" + ) + message(STATUS "Found the Arrow Flight SQL import library: ${ARROW_FLIGHT_SQL_IMPORT_LIB}" + ) + message(STATUS "Found the Arrow Flight SQL static library: ${ARROW_FLIGHT_SQL_STATIC_LIB}" + ) +endif() diff --git a/cpp/cmake_modules/FindSQLite3Alt.cmake b/cpp/cmake_modules/FindSQLite3Alt.cmake new file mode 100644 index 00000000000..73a45f098c6 --- /dev/null +++ b/cpp/cmake_modules/FindSQLite3Alt.cmake @@ -0,0 +1,43 @@ +# 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. + +# Once done this will define +# - FindSQLite3Alt +# +# This module will set the following variables if found: +# SQLite3_INCLUDE_DIRS - SQLite3 include dir. +# SQLite3_LIBRARIES - List of libraries when using SQLite3. +# SQLite3_FOUND - True if SQLite3 found. +# +# Usage of this module as follows: +# find_package(SQLite3Alt) + +find_path(SQLite3_INCLUDE_DIR sqlite3.h) +find_library(SQLite3_LIBRARY NAMES sqlite3) + +# handle the QUIETLY and REQUIRED arguments and set SQLite3_FOUND to TRUE if +# all listed variables are TRUE +include(FindPackageHandleStandardArgs) +find_package_handle_standard_args(SQLite3Alt REQUIRED_VARS SQLite3_LIBRARY + SQLite3_INCLUDE_DIR) + +mark_as_advanced(SQLite3_LIBRARY SQLite3_INCLUDE_DIR) + +if(SQLite3Alt_FOUND) + set(SQLite3_INCLUDE_DIRS ${SQLite3_INCLUDE_DIR}) + set(SQLite3_LIBRARIES ${SQLite3_LIBRARY}) +endif() diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 5736c557bd0..502629a92a4 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -732,6 +732,10 @@ if(ARROW_FLIGHT) add_subdirectory(flight) endif() +if(ARROW_FLIGHT_SQL) + add_subdirectory(flight/sql) +endif() + if(ARROW_HIVESERVER2) add_subdirectory(dbi/hiveserver2) endif() diff --git a/cpp/src/arrow/flight/CMakeLists.txt b/cpp/src/arrow/flight/CMakeLists.txt index 8a3228e5026..55e89b2eb99 100644 --- a/cpp/src/arrow/flight/CMakeLists.txt +++ b/cpp/src/arrow/flight/CMakeLists.txt @@ -25,7 +25,23 @@ if(WIN32) list(APPEND ARROW_FLIGHT_LINK_LIBS ws2_32.lib) endif() -if(ARROW_TEST_LINKAGE STREQUAL "static") +set(ARROW_FLIGHT_TEST_LINKAGE + "${ARROW_TEST_LINKAGE}" + PARENT_SCOPE) +if(Protobuf_USE_STATIC_LIBS) + message(STATUS "Linking Arrow Flight tests statically due to static Protobuf") + set(ARROW_FLIGHT_TEST_LINKAGE + "static" + PARENT_SCOPE) +endif() +if(NOT ARROW_GRPC_USE_SHARED) + message(STATUS "Linking Arrow Flight tests statically due to static gRPC") + set(ARROW_FLIGHT_TEST_LINKAGE + "static" + PARENT_SCOPE) +endif() + +if(ARROW_FLIGHT_TEST_LINKAGE STREQUAL "static") set(ARROW_FLIGHT_TEST_LINK_LIBS arrow_flight_static arrow_flight_testing_static ${ARROW_FLIGHT_STATIC_LINK_LIBS} ${ARROW_TEST_LINK_LIBS}) diff --git a/cpp/src/arrow/flight/sql/ArrowFlightSqlConfig.cmake.in b/cpp/src/arrow/flight/sql/ArrowFlightSqlConfig.cmake.in new file mode 100644 index 00000000000..1658f44f418 --- /dev/null +++ b/cpp/src/arrow/flight/sql/ArrowFlightSqlConfig.cmake.in @@ -0,0 +1,36 @@ +# 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. +# +# This config sets the following variables in your project:: +# +# ArrowFlightSql_FOUND - true if Arrow Flight SQL found on the system +# +# This config sets the following targets in your project:: +# +# arrow_flight_sql_shared - for linked as shared library if shared library is built +# arrow_flight_sql_static - for linked as static library if static library is built + +@PACKAGE_INIT@ + +include(CMakeFindDependencyMacro) +find_dependency(ArrowFlight) + +# Load targets only once. If we load targets multiple times, CMake reports +# already existent target error. +if(NOT (TARGET arrow_flight_sql_shared OR TARGET arrow_flight_sql_static)) + include("${CMAKE_CURRENT_LIST_DIR}/ArrowFlightSqlTargets.cmake") +endif() diff --git a/cpp/src/arrow/flight/sql/CMakeLists.txt b/cpp/src/arrow/flight/sql/CMakeLists.txt new file mode 100644 index 00000000000..4a31f5ba2e2 --- /dev/null +++ b/cpp/src/arrow/flight/sql/CMakeLists.txt @@ -0,0 +1,100 @@ +# 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. + +add_custom_target(arrow_flight_sql) + +arrow_install_all_headers("arrow/flight/sql") + +set(FLIGHT_SQL_PROTO_PATH "${ARROW_SOURCE_DIR}/../format") +set(FLIGHT_SQL_PROTO ${ARROW_SOURCE_DIR}/../format/FlightSql.proto) + +set(FLIGHT_SQL_GENERATED_PROTO_FILES "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.h") + +set(PROTO_DEPENDS ${FLIGHT_SQL_PROTO} ${ARROW_PROTOBUF_LIBPROTOBUF}) + +add_custom_command(OUTPUT ${FLIGHT_SQL_GENERATED_PROTO_FILES} + COMMAND ${ARROW_PROTOBUF_PROTOC} "-I${FLIGHT_SQL_PROTO_PATH}" + "--cpp_out=${CMAKE_CURRENT_BINARY_DIR}" "${FLIGHT_SQL_PROTO}" + DEPENDS ${PROTO_DEPENDS}) + +set_source_files_properties(${FLIGHT_SQL_GENERATED_PROTO_FILES} PROPERTIES GENERATED TRUE) + +add_custom_target(flight_sql_protobuf_gen ALL DEPENDS ${FLIGHT_SQL_GENERATED_PROTO_FILES}) + +set(ARROW_FLIGHT_SQL_SRCS server.cc sql_info_internal.cc client.cc + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc") + +add_arrow_lib(arrow_flight_sql + CMAKE_PACKAGE_NAME + ArrowFlightSql + PKG_CONFIG_NAME + arrow-flight-sql + OUTPUTS + ARROW_FLIGHT_SQL_LIBRARIES + SOURCES + ${ARROW_FLIGHT_SQL_SRCS} + DEPENDENCIES + flight_sql_protobuf_gen + SHARED_LINK_FLAGS + ${ARROW_VERSION_SCRIPT_FLAGS} # Defined in cpp/arrow/CMakeLists.txt + SHARED_LINK_LIBS + arrow_flight_shared + STATIC_LINK_LIBS + arrow_flight_static) + +if(ARROW_FLIGHT_TEST_LINKAGE STREQUAL "static") + set(ARROW_FLIGHT_SQL_TEST_LINK_LIBS + arrow_flight_sql_static arrow_flight_testing_static + ${ARROW_FLIGHT_STATIC_LINK_LIBS} ${ARROW_TEST_LINK_LIBS}) +else() + set(ARROW_FLIGHT_SQL_TEST_LINK_LIBS arrow_flight_sql_shared arrow_flight_testing_shared + ${ARROW_TEST_LINK_LIBS}) +endif() + +# Build test server for unit tests +if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) + find_package(SQLite3Alt REQUIRED) + + set(ARROW_FLIGHT_SQL_TEST_SERVER_SRCS + example/sqlite_sql_info.cc + example/sqlite_statement.cc + example/sqlite_statement_batch_reader.cc + example/sqlite_server.cc + example/sqlite_tables_schema_batch_reader.cc) + + add_arrow_test(flight_sql_test + SOURCES + client_test.cc + server_test.cc + ${ARROW_FLIGHT_SQL_TEST_SERVER_SRCS} + STATIC_LINK_LIBS + ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} + ${SQLite3_LIBRARIES} + LABELS + "arrow_flight_sql") + + add_executable(flight_sql_test_server test_server_cli.cc + ${ARROW_FLIGHT_SQL_TEST_SERVER_SRCS}) + target_link_libraries(flight_sql_test_server + PRIVATE ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} ${GFLAGS_LIBRARIES} + ${SQLite3_LIBRARIES}) + + add_executable(flight_sql_test_app test_app_cli.cc) + target_link_libraries(flight_sql_test_app PRIVATE ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} + ${GFLAGS_LIBRARIES}) +endif() diff --git a/cpp/src/arrow/flight/sql/api.h b/cpp/src/arrow/flight/sql/api.h new file mode 100644 index 00000000000..3b909eedf29 --- /dev/null +++ b/cpp/src/arrow/flight/sql/api.h @@ -0,0 +1,20 @@ +// 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. + +#pragma once + +#include "arrow/flight/sql/client.h" diff --git a/cpp/src/arrow/flight/sql/arrow-flight-sql.pc.in b/cpp/src/arrow/flight/sql/arrow-flight-sql.pc.in new file mode 100644 index 00000000000..6d4eab0b4a0 --- /dev/null +++ b/cpp/src/arrow/flight/sql/arrow-flight-sql.pc.in @@ -0,0 +1,25 @@ +# 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. + +libdir=@CMAKE_INSTALL_FULL_LIBDIR@ +includedir=@CMAKE_INSTALL_FULL_INCLUDEDIR@ + +Name: Apache Arrow Flight SQL +Description: Apache Arrow Flight SQL extension +Version: @ARROW_VERSION@ +Requires: arrow-flight +Libs: -L${libdir} -larrow_flight_sql diff --git a/cpp/src/arrow/flight/sql/client.cc b/cpp/src/arrow/flight/sql/client.cc new file mode 100644 index 00000000000..50a5777cd9c --- /dev/null +++ b/cpp/src/arrow/flight/sql/client.cc @@ -0,0 +1,425 @@ +// 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. + +#include "arrow/flight/sql/client.h" + +#include + +#include "arrow/buffer.h" +#include "arrow/flight/sql/FlightSql.pb.h" +#include "arrow/flight/types.h" +#include "arrow/io/memory.h" +#include "arrow/ipc/reader.h" +#include "arrow/result.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/util/logging.h" + +namespace flight_sql_pb = arrow::flight::protocol::sql; + +namespace arrow { +namespace flight { +namespace sql { + +FlightSqlClient::FlightSqlClient(std::shared_ptr client) + : impl_(std::move(client)) {} + +PreparedStatement::PreparedStatement(FlightSqlClient* client, std::string handle, + std::shared_ptr dataset_schema, + std::shared_ptr parameter_schema, + FlightCallOptions options) + : client_(client), + options_(std::move(options)), + handle_(std::move(handle)), + dataset_schema_(std::move(dataset_schema)), + parameter_schema_(std::move(parameter_schema)), + is_closed_(false) {} + +PreparedStatement::~PreparedStatement() { + if (IsClosed()) return; + + const Status status = Close(); + if (!status.ok()) { + ARROW_LOG(ERROR) << "Failed to delete PreparedStatement: " << status.ToString(); + } +} + +inline FlightDescriptor GetFlightDescriptorForCommand( + const google::protobuf::Message& command) { + google::protobuf::Any any; + any.PackFrom(command); + + const std::string& string = any.SerializeAsString(); + return FlightDescriptor::Command(string); +} + +arrow::Result> GetFlightInfoForCommand( + FlightSqlClient& client, const FlightCallOptions& options, + const google::protobuf::Message& command) { + const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); + + ARROW_ASSIGN_OR_RAISE(auto flight_info, client.GetFlightInfo(options, descriptor)); + return std::move(flight_info); +} + +arrow::Result> FlightSqlClient::Execute( + const FlightCallOptions& options, const std::string& query) { + flight_sql_pb::CommandStatementQuery command; + command.set_query(query); + + return GetFlightInfoForCommand(*this, options, command); +} + +arrow::Result FlightSqlClient::ExecuteUpdate(const FlightCallOptions& options, + const std::string& query) { + flight_sql_pb::CommandStatementUpdate command; + command.set_query(query); + + const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); + + std::unique_ptr writer; + std::unique_ptr reader; + + ARROW_RETURN_NOT_OK(DoPut(options, descriptor, NULLPTR, &writer, &reader)); + + std::shared_ptr metadata; + + ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata)); + + flight_sql_pb::DoPutUpdateResult doPutUpdateResult; + + flight_sql_pb::DoPutUpdateResult result; + if (!result.ParseFromArray(metadata->data(), static_cast(metadata->size()))) { + return Status::Invalid("Unable to parse DoPutUpdateResult object."); + } + + return result.record_count(); +} + +arrow::Result> FlightSqlClient::GetCatalogs( + const FlightCallOptions& options) { + flight_sql_pb::CommandGetCatalogs command; + + return GetFlightInfoForCommand(*this, options, command); +} + +arrow::Result> FlightSqlClient::GetDbSchemas( + const FlightCallOptions& options, const std::string* catalog, + const std::string* db_schema_filter_pattern) { + flight_sql_pb::CommandGetDbSchemas command; + if (catalog != NULLPTR) { + command.set_catalog(*catalog); + } + if (db_schema_filter_pattern != NULLPTR) { + command.set_db_schema_filter_pattern(*db_schema_filter_pattern); + } + + return GetFlightInfoForCommand(*this, options, command); +} + +arrow::Result> FlightSqlClient::GetTables( + const FlightCallOptions& options, const std::string* catalog, + const std::string* db_schema_filter_pattern, const std::string* table_filter_pattern, + bool include_schema, const std::vector* table_types) { + flight_sql_pb::CommandGetTables command; + + if (catalog != NULLPTR) { + command.set_catalog(*catalog); + } + + if (db_schema_filter_pattern != NULLPTR) { + command.set_db_schema_filter_pattern(*db_schema_filter_pattern); + } + + if (table_filter_pattern != NULLPTR) { + command.set_table_name_filter_pattern(*table_filter_pattern); + } + + command.set_include_schema(include_schema); + + if (table_types != NULLPTR) { + for (const std::string& table_type : *table_types) { + command.add_table_types(table_type); + } + } + + return GetFlightInfoForCommand(*this, options, command); +} + +arrow::Result> FlightSqlClient::GetPrimaryKeys( + const FlightCallOptions& options, const TableRef& table_ref) { + flight_sql_pb::CommandGetPrimaryKeys command; + + if (table_ref.catalog.has_value()) { + command.set_catalog(table_ref.catalog.value()); + } + + if (table_ref.db_schema.has_value()) { + command.set_db_schema(table_ref.db_schema.value()); + } + + command.set_table(table_ref.table); + + return GetFlightInfoForCommand(*this, options, command); +} + +arrow::Result> FlightSqlClient::GetExportedKeys( + const FlightCallOptions& options, const TableRef& table_ref) { + flight_sql_pb::CommandGetExportedKeys command; + + if (table_ref.catalog.has_value()) { + command.set_catalog(table_ref.catalog.value()); + } + + if (table_ref.db_schema.has_value()) { + command.set_db_schema(table_ref.db_schema.value()); + } + + command.set_table(table_ref.table); + + return GetFlightInfoForCommand(*this, options, command); +} + +arrow::Result> FlightSqlClient::GetImportedKeys( + const FlightCallOptions& options, const TableRef& table_ref) { + flight_sql_pb::CommandGetImportedKeys command; + + if (table_ref.catalog.has_value()) { + command.set_catalog(table_ref.catalog.value()); + } + + if (table_ref.db_schema.has_value()) { + command.set_db_schema(table_ref.db_schema.value()); + } + + command.set_table(table_ref.table); + + return GetFlightInfoForCommand(*this, options, command); +} + +arrow::Result> FlightSqlClient::GetCrossReference( + const FlightCallOptions& options, const TableRef& pk_table_ref, + const TableRef& fk_table_ref) { + flight_sql_pb::CommandGetCrossReference command; + + if (pk_table_ref.catalog.has_value()) { + command.set_pk_catalog(pk_table_ref.catalog.value()); + } + if (pk_table_ref.db_schema.has_value()) { + command.set_pk_db_schema(pk_table_ref.db_schema.value()); + } + command.set_pk_table(pk_table_ref.table); + + if (fk_table_ref.catalog.has_value()) { + command.set_fk_catalog(fk_table_ref.catalog.value()); + } + if (fk_table_ref.db_schema.has_value()) { + command.set_fk_db_schema(fk_table_ref.db_schema.value()); + } + command.set_fk_table(fk_table_ref.table); + + return GetFlightInfoForCommand(*this, options, command); +} + +arrow::Result> FlightSqlClient::GetTableTypes( + const FlightCallOptions& options) { + flight_sql_pb::CommandGetTableTypes command; + + return GetFlightInfoForCommand(*this, options, command); +} + +arrow::Result> FlightSqlClient::DoGet( + const FlightCallOptions& options, const Ticket& ticket) { + std::unique_ptr stream; + ARROW_RETURN_NOT_OK(DoGet(options, ticket, &stream)); + + return std::move(stream); +} + +arrow::Result> FlightSqlClient::Prepare( + const FlightCallOptions& options, const std::string& query) { + google::protobuf::Any command; + flight_sql_pb::ActionCreatePreparedStatementRequest request; + request.set_query(query); + command.PackFrom(request); + + Action action; + action.type = "CreatePreparedStatement"; + action.body = Buffer::FromString(command.SerializeAsString()); + + std::unique_ptr results; + + ARROW_RETURN_NOT_OK(DoAction(options, action, &results)); + + std::unique_ptr result; + ARROW_RETURN_NOT_OK(results->Next(&result)); + + google::protobuf::Any prepared_result; + + std::shared_ptr message = std::move(result->body); + if (!prepared_result.ParseFromArray(message->data(), + static_cast(message->size()))) { + return Status::Invalid("Unable to parse packed ActionCreatePreparedStatementResult"); + } + + flight_sql_pb::ActionCreatePreparedStatementResult prepared_statement_result; + + if (!prepared_result.UnpackTo(&prepared_statement_result)) { + return Status::Invalid("Unable to unpack ActionCreatePreparedStatementResult"); + } + + const std::string& serialized_dataset_schema = + prepared_statement_result.dataset_schema(); + const std::string& serialized_parameter_schema = + prepared_statement_result.parameter_schema(); + + std::shared_ptr dataset_schema; + if (!serialized_dataset_schema.empty()) { + io::BufferReader dataset_schema_reader(serialized_dataset_schema); + ipc::DictionaryMemo in_memo; + ARROW_ASSIGN_OR_RAISE(dataset_schema, ReadSchema(&dataset_schema_reader, &in_memo)); + } + std::shared_ptr parameter_schema; + if (!serialized_parameter_schema.empty()) { + io::BufferReader parameter_schema_reader(serialized_parameter_schema); + ipc::DictionaryMemo in_memo; + ARROW_ASSIGN_OR_RAISE(parameter_schema, + ReadSchema(¶meter_schema_reader, &in_memo)); + } + auto handle = prepared_statement_result.prepared_statement_handle(); + + return std::make_shared(this, handle, dataset_schema, + parameter_schema, options); +} + +arrow::Result> PreparedStatement::Execute() { + if (is_closed_) { + return Status::Invalid("Statement already closed."); + } + + flight_sql_pb::CommandPreparedStatementQuery execute_query_command; + + execute_query_command.set_prepared_statement_handle(handle_); + + google::protobuf::Any any; + any.PackFrom(execute_query_command); + + const std::string& string = any.SerializeAsString(); + const FlightDescriptor descriptor = FlightDescriptor::Command(string); + + if (parameter_binding_ && parameter_binding_->num_rows() > 0) { + std::unique_ptr writer; + std::unique_ptr reader; + ARROW_RETURN_NOT_OK(client_->DoPut(options_, descriptor, parameter_binding_->schema(), + &writer, &reader)); + + ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*parameter_binding_)); + ARROW_RETURN_NOT_OK(writer->DoneWriting()); + // Wait for the server to ack the result + std::shared_ptr buffer; + ARROW_RETURN_NOT_OK(reader->ReadMetadata(&buffer)); + } + + ARROW_ASSIGN_OR_RAISE(auto flight_info, client_->GetFlightInfo(options_, descriptor)); + return std::move(flight_info); +} + +arrow::Result PreparedStatement::ExecuteUpdate() { + if (is_closed_) { + return Status::Invalid("Statement already closed."); + } + + flight_sql_pb::CommandPreparedStatementUpdate command; + command.set_prepared_statement_handle(handle_); + const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); + std::unique_ptr writer; + std::unique_ptr reader; + + if (parameter_binding_ && parameter_binding_->num_rows() > 0) { + ARROW_RETURN_NOT_OK(client_->DoPut(options_, descriptor, parameter_binding_->schema(), + &writer, &reader)); + ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*parameter_binding_)); + } else { + const std::shared_ptr schema = arrow::schema({}); + ARROW_RETURN_NOT_OK(client_->DoPut(options_, descriptor, schema, &writer, &reader)); + const auto& record_batch = + arrow::RecordBatch::Make(schema, 0, (std::vector>){}); + ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*record_batch)); + } + + ARROW_RETURN_NOT_OK(writer->DoneWriting()); + std::shared_ptr metadata; + ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata)); + ARROW_RETURN_NOT_OK(writer->Close()); + + flight_sql_pb::DoPutUpdateResult result; + if (!result.ParseFromArray(metadata->data(), static_cast(metadata->size()))) { + return Status::Invalid("Unable to parse DoPutUpdateResult object."); + } + + return result.record_count(); +} + +Status PreparedStatement::SetParameters(std::shared_ptr parameter_binding) { + parameter_binding_ = std::move(parameter_binding); + + return Status::OK(); +} + +bool PreparedStatement::IsClosed() const { return is_closed_; } + +std::shared_ptr PreparedStatement::dataset_schema() const { + return dataset_schema_; +} + +std::shared_ptr PreparedStatement::parameter_schema() const { + return parameter_schema_; +} + +Status PreparedStatement::Close() { + if (is_closed_) { + return Status::Invalid("Statement already closed."); + } + google::protobuf::Any command; + flight_sql_pb::ActionClosePreparedStatementRequest request; + request.set_prepared_statement_handle(handle_); + + command.PackFrom(request); + + Action action; + action.type = "ClosePreparedStatement"; + action.body = Buffer::FromString(command.SerializeAsString()); + + std::unique_ptr results; + + ARROW_RETURN_NOT_OK(client_->DoAction(options_, action, &results)); + + is_closed_ = true; + + return Status::OK(); +} + +arrow::Result> FlightSqlClient::GetSqlInfo( + const FlightCallOptions& options, const std::vector& sql_info) { + flight_sql_pb::CommandGetSqlInfo command; + for (const int& info : sql_info) command.add_info(info); + + return GetFlightInfoForCommand(*this, options, command); +} + +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/client.h b/cpp/src/arrow/flight/sql/client.h new file mode 100644 index 00000000000..5bf1b3e64a0 --- /dev/null +++ b/cpp/src/arrow/flight/sql/client.h @@ -0,0 +1,247 @@ +// 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. + +#pragma once + +#include +#include + +#include "arrow/flight/client.h" +#include "arrow/flight/sql/types.h" +#include "arrow/flight/types.h" +#include "arrow/result.h" +#include "arrow/status.h" + +namespace arrow { +namespace flight { +namespace sql { + +class PreparedStatement; + +/// \brief Flight client with Flight SQL semantics. +class ARROW_EXPORT FlightSqlClient { + friend class PreparedStatement; + + private: + std::shared_ptr impl_; + + public: + explicit FlightSqlClient(std::shared_ptr client); + + virtual ~FlightSqlClient() = default; + + /// \brief Execute a query on the server. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] query The query to be executed in the UTF-8 format. + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> Execute(const FlightCallOptions& options, + const std::string& query); + + /// \brief Execute an update query on the server. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] query The query to be executed in the UTF-8 format. + /// \return The quantity of rows affected by the operation. + arrow::Result ExecuteUpdate(const FlightCallOptions& options, + const std::string& query); + + /// \brief Request a list of catalogs. + /// \param[in] options RPC-layer hints for this call. + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetCatalogs( + const FlightCallOptions& options); + + /// \brief Request a list of database schemas. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] catalog The catalog. + /// \param[in] db_schema_filter_pattern The schema filter pattern. + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetDbSchemas( + const FlightCallOptions& options, const std::string* catalog, + const std::string* db_schema_filter_pattern); + + /// \brief Given a flight ticket and schema, request to be sent the + /// stream. Returns record batch stream reader + /// \param[in] options Per-RPC options + /// \param[in] ticket The flight ticket to use + /// \return The returned RecordBatchReader + arrow::Result> DoGet( + const FlightCallOptions& options, const Ticket& ticket); + + /// \brief Request a list of tables. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] catalog The catalog. + /// \param[in] db_schema_filter_pattern The schema filter pattern. + /// \param[in] table_filter_pattern The table filter pattern. + /// \param[in] include_schema True to include the schema upon return, + /// false to not include the schema. + /// \param[in] table_types The table types to include. + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetTables( + const FlightCallOptions& options, const std::string* catalog, + const std::string* db_schema_filter_pattern, + const std::string* table_filter_pattern, bool include_schema, + const std::vector* table_types); + + /// \brief Request the primary keys for a table. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] table_ref The table reference. + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetPrimaryKeys( + const FlightCallOptions& options, const TableRef& table_ref); + + /// \brief Retrieves a description about the foreign key columns that reference the + /// primary key columns of the given table. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] table_ref The table reference. + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetExportedKeys( + const FlightCallOptions& options, const TableRef& table_ref); + + /// \brief Retrieves the foreign key columns for the given table. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] table_ref The table reference. + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetImportedKeys( + const FlightCallOptions& options, const TableRef& table_ref); + + /// \brief Retrieves 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). + /// \param[in] options RPC-layer hints for this call. + /// \param[in] pk_table_ref The table reference that exports the key. + /// \param[in] fk_table_ref The table reference that imports the key. + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetCrossReference( + const FlightCallOptions& options, const TableRef& pk_table_ref, + const TableRef& fk_table_ref); + + /// \brief Request a list of table types. + /// \param[in] options RPC-layer hints for this call. + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetTableTypes( + const FlightCallOptions& options); + + /// \brief Request a list of SQL information. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] sql_info the SQL info required. + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetSqlInfo(const FlightCallOptions& options, + const std::vector& sql_info); + + /// \brief Create a prepared statement object. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] query The query that will be executed. + /// \return The created prepared statement. + arrow::Result> Prepare( + const FlightCallOptions& options, const std::string& query); + + /// \brief Retrieve the FlightInfo. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] descriptor The flight descriptor. + /// \return The flight info with the metadata. + // NOTE: This is public because it is been used by the anonymous + // function GetFlightInfoForCommand. + virtual arrow::Result> GetFlightInfo( + const FlightCallOptions& options, const FlightDescriptor& descriptor) { + std::unique_ptr info; + ARROW_RETURN_NOT_OK(impl_->GetFlightInfo(options, descriptor, &info)); + + return info; + } + + protected: + virtual Status DoPut(const FlightCallOptions& options, + const FlightDescriptor& descriptor, + const std::shared_ptr& schema, + std::unique_ptr* stream, + std::unique_ptr* reader) { + return impl_->DoPut(options, descriptor, schema, stream, reader); + } + + virtual Status DoGet(const FlightCallOptions& options, const Ticket& ticket, + std::unique_ptr* stream) { + return impl_->DoGet(options, ticket, stream); + } + + virtual Status DoAction(const FlightCallOptions& options, const Action& action, + std::unique_ptr* results) { + return impl_->DoAction(options, action, results); + } +}; + +/// \brief PreparedStatement class from flight sql. +class ARROW_EXPORT PreparedStatement { + FlightSqlClient* client_; + FlightCallOptions options_; + std::string handle_; + std::shared_ptr dataset_schema_; + std::shared_ptr parameter_schema_; + std::shared_ptr parameter_binding_; + bool is_closed_; + + public: + /// \brief Constructor for the PreparedStatement class. + /// \param[in] client Client object used to make the RPC requests. + /// \param[in] handle Handle for this prepared statement. + /// \param[in] dataset_schema Schema of the resulting dataset. + /// \param[in] parameter_schema Schema of the parameters (if any). + /// \param[in] options RPC-layer hints for this call. + PreparedStatement(FlightSqlClient* client, std::string handle, + std::shared_ptr dataset_schema, + std::shared_ptr parameter_schema, FlightCallOptions options); + + /// \brief Default destructor for the PreparedStatement class. + /// The destructor will call the Close method from the class in order, + /// to send a request to close the PreparedStatement. + /// NOTE: It is best to explicitly close the PreparedStatement, otherwise + /// errors can't be caught. + ~PreparedStatement(); + + /// \brief Executes the prepared statement query on the server. + /// \return A FlightInfo object representing the stream(s) to fetch. + arrow::Result> Execute(); + + /// \brief Executes the prepared statement update query on the server. + /// \return The number of rows affected. + arrow::Result ExecuteUpdate(); + + /// \brief Retrieve the parameter schema from the query. + /// \return The parameter schema from the query. + std::shared_ptr parameter_schema() const; + + /// \brief Retrieve the ResultSet schema from the query. + /// \return The ResultSet schema from the query. + std::shared_ptr dataset_schema() const; + + /// \brief Set a RecordBatch that contains the parameters that will be bind. + /// \param parameter_binding The parameters that will be bind. + /// \return Status. + Status SetParameters(std::shared_ptr parameter_binding); + + /// \brief Close the prepared statement, so that this PreparedStatement can not used + /// anymore and server can free up any resources. + /// \return Status. + Status Close(); + + /// \brief Check if the prepared statement is closed. + /// \return The state of the prepared statement. + bool IsClosed() const; +}; + +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/client_test.cc b/cpp/src/arrow/flight/sql/client_test.cc new file mode 100644 index 00000000000..8c0c8333074 --- /dev/null +++ b/cpp/src/arrow/flight/sql/client_test.cc @@ -0,0 +1,515 @@ +// 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. + +#include "arrow/flight/client.h" + +#include +#include +#include + +#include + +#include "arrow/flight/sql/FlightSql.pb.h" +#include "arrow/flight/sql/api.h" +#include "arrow/testing/gtest_util.h" + +namespace pb = arrow::flight::protocol; +using ::testing::_; +using ::testing::Ref; + +namespace arrow { +namespace flight { +namespace sql { + +class FlightSqlClientMock : public FlightSqlClient { + public: + FlightSqlClientMock() : FlightSqlClient(nullptr) {} + + ~FlightSqlClientMock() = default; + + MOCK_METHOD(arrow::Result>, GetFlightInfo, + (const FlightCallOptions&, const FlightDescriptor&)); + MOCK_METHOD(Status, DoGet, + (const FlightCallOptions& options, const Ticket& ticket, + std::unique_ptr* stream)); + MOCK_METHOD(Status, DoPut, + (const FlightCallOptions&, const FlightDescriptor&, + const std::shared_ptr& schema, + std::unique_ptr*, + std::unique_ptr*)); + MOCK_METHOD(Status, DoAction, + (const FlightCallOptions& options, const Action& action, + std::unique_ptr* results)); +}; + +class TestFlightSqlClient : public ::testing::Test { + protected: + FlightSqlClientMock sql_client_; + FlightCallOptions call_options_; + + void SetUp() override {} + + void TearDown() override {} +}; + +class FlightMetadataReaderMock : public FlightMetadataReader { + public: + std::shared_ptr* buffer; + + explicit FlightMetadataReaderMock(std::shared_ptr* buffer) { + this->buffer = buffer; + } + + Status ReadMetadata(std::shared_ptr* out) override { + *out = *buffer; + return Status::OK(); + } +}; + +class FlightStreamWriterMock : public FlightStreamWriter { + public: + FlightStreamWriterMock() = default; + + Status DoneWriting() override { return Status::OK(); } + + Status WriteMetadata(std::shared_ptr app_metadata) override { + return Status::OK(); + } + + Status Begin(const std::shared_ptr& schema, + const ipc::IpcWriteOptions& options) override { + return Status::OK(); + } + + Status Begin(const std::shared_ptr& schema) override { + return MetadataRecordBatchWriter::Begin(schema); + } + + ipc::WriteStats stats() const override { return ipc::WriteStats(); } + + Status WriteWithMetadata(const RecordBatch& batch, + std::shared_ptr app_metadata) override { + return Status::OK(); + } + + Status Close() override { return Status::OK(); } + + Status WriteRecordBatch(const RecordBatch& batch) override { return Status::OK(); } +}; + +FlightDescriptor getDescriptor(google::protobuf::Message& command) { + google::protobuf::Any any; + any.PackFrom(command); + + const std::string& string = any.SerializeAsString(); + return FlightDescriptor::Command(string); +} + +auto ReturnEmptyFlightInfo = [](const FlightCallOptions& options, + const FlightDescriptor& descriptor) { + std::unique_ptr flight_info; + return flight_info; +}; + +TEST_F(TestFlightSqlClient, TestGetCatalogs) { + pb::sql::CommandGetCatalogs command; + FlightDescriptor descriptor = getDescriptor(command); + + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); + + ASSERT_OK(sql_client_.GetCatalogs(call_options_)); +} + +TEST_F(TestFlightSqlClient, TestGetDbSchemas) { + std::string schema_filter_pattern = "schema_filter_pattern"; + std::string catalog = "catalog"; + + pb::sql::CommandGetDbSchemas command; + command.set_catalog(catalog); + command.set_db_schema_filter_pattern(schema_filter_pattern); + FlightDescriptor descriptor = getDescriptor(command); + + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); + + ASSERT_OK(sql_client_.GetDbSchemas(call_options_, &catalog, &schema_filter_pattern)); +} + +TEST_F(TestFlightSqlClient, TestGetTables) { + std::string catalog = "catalog"; + std::string schema_filter_pattern = "schema_filter_pattern"; + std::string table_name_filter_pattern = "table_name_filter_pattern"; + bool include_schema = true; + std::vector table_types = {"type1", "type2"}; + + pb::sql::CommandGetTables command; + command.set_catalog(catalog); + command.set_db_schema_filter_pattern(schema_filter_pattern); + command.set_table_name_filter_pattern(table_name_filter_pattern); + command.set_include_schema(include_schema); + for (const std::string& table_type : table_types) { + command.add_table_types(table_type); + } + FlightDescriptor descriptor = getDescriptor(command); + + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); + + ASSERT_OK(sql_client_.GetTables(call_options_, &catalog, &schema_filter_pattern, + &table_name_filter_pattern, include_schema, + &table_types)); +} + +TEST_F(TestFlightSqlClient, TestGetTableTypes) { + pb::sql::CommandGetTableTypes command; + FlightDescriptor descriptor = getDescriptor(command); + + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); + + ASSERT_OK(sql_client_.GetTableTypes(call_options_)); +} + +TEST_F(TestFlightSqlClient, TestGetExported) { + std::string catalog = "catalog"; + std::string schema = "schema"; + std::string table = "table"; + + pb::sql::CommandGetExportedKeys command; + command.set_catalog(catalog); + command.set_db_schema(schema); + command.set_table(table); + FlightDescriptor descriptor = getDescriptor(command); + + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); + + TableRef table_ref = {util::make_optional(catalog), util::make_optional(schema), table}; + ASSERT_OK(sql_client_.GetExportedKeys(call_options_, table_ref)); +} + +TEST_F(TestFlightSqlClient, TestGetImported) { + std::string catalog = "catalog"; + std::string schema = "schema"; + std::string table = "table"; + + pb::sql::CommandGetImportedKeys command; + command.set_catalog(catalog); + command.set_db_schema(schema); + command.set_table(table); + FlightDescriptor descriptor = getDescriptor(command); + + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); + + TableRef table_ref = {util::make_optional(catalog), util::make_optional(schema), table}; + ASSERT_OK(sql_client_.GetImportedKeys(call_options_, table_ref)); +} + +TEST_F(TestFlightSqlClient, TestGetPrimary) { + std::string catalog = "catalog"; + std::string schema = "schema"; + std::string table = "table"; + + pb::sql::CommandGetPrimaryKeys command; + command.set_catalog(catalog); + command.set_db_schema(schema); + command.set_table(table); + FlightDescriptor descriptor = getDescriptor(command); + + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); + + TableRef table_ref = {util::make_optional(catalog), util::make_optional(schema), table}; + ASSERT_OK(sql_client_.GetPrimaryKeys(call_options_, table_ref)); +} + +TEST_F(TestFlightSqlClient, TestGetCrossReference) { + std::string pk_catalog = "pk_catalog"; + std::string pk_schema = "pk_schema"; + std::string pk_table = "pk_table"; + std::string fk_catalog = "fk_catalog"; + std::string fk_schema = "fk_schema"; + std::string fk_table = "fk_table"; + + pb::sql::CommandGetCrossReference command; + command.set_pk_catalog(pk_catalog); + command.set_pk_db_schema(pk_schema); + command.set_pk_table(pk_table); + command.set_fk_catalog(fk_catalog); + command.set_fk_db_schema(fk_schema); + command.set_fk_table(fk_table); + FlightDescriptor descriptor = getDescriptor(command); + + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); + + TableRef pk_table_ref = {util::make_optional(pk_catalog), + util::make_optional(pk_schema), pk_table}; + TableRef fk_table_ref = {util::make_optional(fk_catalog), + util::make_optional(fk_schema), fk_table}; + ASSERT_OK(sql_client_.GetCrossReference(call_options_, pk_table_ref, fk_table_ref)); +} + +TEST_F(TestFlightSqlClient, TestExecute) { + std::string query = "query"; + + pb::sql::CommandStatementQuery command; + command.set_query(query); + FlightDescriptor descriptor = getDescriptor(command); + + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); + + ASSERT_OK(sql_client_.Execute(call_options_, query)); +} + +TEST_F(TestFlightSqlClient, TestPreparedStatementExecute) { + const std::string query = "query"; + + ON_CALL(sql_client_, DoAction) + .WillByDefault([](const FlightCallOptions& options, const Action& action, + std::unique_ptr* results) { + google::protobuf::Any command; + + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + + prepared_statement_result.set_prepared_statement_handle("query"); + + command.PackFrom(prepared_statement_result); + + *results = std::unique_ptr(new SimpleResultStream( + {Result{Buffer::FromString(command.SerializeAsString())}})); + + return Status::OK(); + }); + + EXPECT_CALL(sql_client_, DoAction(_, _, _)).Times(2); + + ASSERT_OK_AND_ASSIGN(auto prepared_statement, + sql_client_.Prepare(call_options_, query)); + + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(_, _)); + + ASSERT_OK(prepared_statement->Execute()); +} + +TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { + const std::string query = "query"; + + ON_CALL(sql_client_, DoAction) + .WillByDefault([](const FlightCallOptions& options, const Action& action, + std::unique_ptr* results) { + google::protobuf::Any command; + + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + + prepared_statement_result.set_prepared_statement_handle("query"); + + auto schema = arrow::schema({arrow::field("id", int64())}); + + std::shared_ptr schema_buffer; + const arrow::Result>& result = + arrow::ipc::SerializeSchema(*schema); + + ARROW_ASSIGN_OR_RAISE(schema_buffer, result); + + prepared_statement_result.set_parameter_schema(schema_buffer->ToString()); + + command.PackFrom(prepared_statement_result); + + *results = std::unique_ptr(new SimpleResultStream( + {Result{Buffer::FromString(command.SerializeAsString())}})); + + return Status::OK(); + }); + + std::shared_ptr buffer_ptr; + ON_CALL(sql_client_, DoPut) + .WillByDefault([&buffer_ptr](const FlightCallOptions& options, + const FlightDescriptor& descriptor1, + const std::shared_ptr& schema, + std::unique_ptr* writer, + std::unique_ptr* reader) { + writer->reset(new FlightStreamWriterMock()); + reader->reset(new FlightMetadataReaderMock(&buffer_ptr)); + + return Status::OK(); + }); + + EXPECT_CALL(sql_client_, DoAction(_, _, _)).Times(2); + EXPECT_CALL(sql_client_, DoPut(_, _, _, _, _)); + + ASSERT_OK_AND_ASSIGN(auto prepared_statement, + sql_client_.Prepare(call_options_, query)); + + auto parameter_schema = prepared_statement->parameter_schema(); + + auto result = RecordBatchFromJSON(parameter_schema, "[[1]]"); + ASSERT_OK(prepared_statement->SetParameters(result)); + + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(_, _)); + + ASSERT_OK(prepared_statement->Execute()); +} + +TEST_F(TestFlightSqlClient, TestExecuteUpdate) { + std::string query = "query"; + + pb::sql::CommandStatementUpdate command; + + command.set_query(query); + + google::protobuf::Any any; + any.PackFrom(command); + + const FlightDescriptor& descriptor = FlightDescriptor::Command(any.SerializeAsString()); + + pb::sql::DoPutUpdateResult doPutUpdateResult; + doPutUpdateResult.set_record_count(100); + const std::string& string = doPutUpdateResult.SerializeAsString(); + + auto buffer_ptr = std::make_shared( + reinterpret_cast(string.data()), doPutUpdateResult.ByteSizeLong()); + + ON_CALL(sql_client_, DoPut) + .WillByDefault([&buffer_ptr](const FlightCallOptions& options, + const FlightDescriptor& descriptor1, + const std::shared_ptr& schema, + std::unique_ptr* writer, + std::unique_ptr* reader) { + reader->reset(new FlightMetadataReaderMock(&buffer_ptr)); + + return Status::OK(); + }); + + std::unique_ptr flight_info; + std::unique_ptr writer; + std::unique_ptr reader; + EXPECT_CALL(sql_client_, DoPut(Ref(call_options_), descriptor, _, _, _)); + + ASSERT_OK_AND_ASSIGN(auto num_rows, sql_client_.ExecuteUpdate(call_options_, query)); + + ASSERT_EQ(num_rows, 100); +} + +TEST_F(TestFlightSqlClient, TestGetSqlInfo) { + std::vector sql_info{pb::sql::SqlInfo::FLIGHT_SQL_SERVER_NAME, + pb::sql::SqlInfo::FLIGHT_SQL_SERVER_VERSION, + pb::sql::SqlInfo::FLIGHT_SQL_SERVER_ARROW_VERSION}; + pb::sql::CommandGetSqlInfo command; + + for (const auto& info : sql_info) command.add_info(info); + google::protobuf::Any any; + any.PackFrom(command); + const FlightDescriptor& descriptor = FlightDescriptor::Command(any.SerializeAsString()); + + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); + + ASSERT_OK(sql_client_.GetSqlInfo(call_options_, sql_info)); +} + +template +inline void AssertTestPreparedStatementExecuteUpdateOk( + Func func, const std::shared_ptr* schema, FlightSqlClientMock& sql_client_) { + const std::string query = "SELECT * FROM IRRELEVANT"; + int64_t expected_rows = 100L; + pb::sql::DoPutUpdateResult result; + result.set_record_count(expected_rows); + + ON_CALL(sql_client_, DoAction) + .WillByDefault([&query, &schema](const FlightCallOptions& options, + const Action& action, + std::unique_ptr* results) { + google::protobuf::Any command; + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + + prepared_statement_result.set_prepared_statement_handle(query); + + if (schema != NULLPTR) { + std::shared_ptr schema_buffer; + const arrow::Result>& result = + arrow::ipc::SerializeSchema(**schema); + + ARROW_ASSIGN_OR_RAISE(schema_buffer, result); + prepared_statement_result.set_parameter_schema(schema_buffer->ToString()); + } + + command.PackFrom(prepared_statement_result); + *results = std::unique_ptr(new SimpleResultStream( + {Result{Buffer::FromString(command.SerializeAsString())}})); + + return Status::OK(); + }); + EXPECT_CALL(sql_client_, DoAction(_, _, _)).Times(2); + + auto buffer = Buffer::FromString(result.SerializeAsString()); + ON_CALL(sql_client_, DoPut) + .WillByDefault([&buffer](const FlightCallOptions& options, + const FlightDescriptor& descriptor1, + const std::shared_ptr& schema, + std::unique_ptr* writer, + std::unique_ptr* reader) { + reader->reset(new FlightMetadataReaderMock(&buffer)); + writer->reset(new FlightStreamWriterMock()); + return Status::OK(); + }); + if (schema == NULLPTR) { + EXPECT_CALL(sql_client_, DoPut(_, _, _, _, _)); + } else { + EXPECT_CALL(sql_client_, DoPut(_, _, *schema, _, _)); + } + + ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client_.Prepare({}, query)); + func(prepared_statement, sql_client_, schema, expected_rows); + ASSERT_OK_AND_ASSIGN(auto rows, prepared_statement->ExecuteUpdate()); + ASSERT_EQ(expected_rows, rows); + ASSERT_OK(prepared_statement->Close()); +} + +TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateNoParameterBinding) { + AssertTestPreparedStatementExecuteUpdateOk( + [](const std::shared_ptr& prepared_statement, + FlightSqlClient& sql_client_, const std::shared_ptr* schema, + const int64_t& row_count) {}, + NULLPTR, sql_client_); +} + +TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding) { + const auto schema = arrow::schema( + {arrow::field("field0", arrow::utf8()), arrow::field("field1", arrow::uint8())}); + AssertTestPreparedStatementExecuteUpdateOk( + [](const std::shared_ptr& prepared_statement, + FlightSqlClient& sql_client_, const std::shared_ptr* schema, + const int64_t& row_count) { + auto string_array = + ArrayFromJSON(utf8(), R"(["Lorem", "Ipsum", "Foo", "Bar", "Baz"])"); + auto uint8_array = ArrayFromJSON(uint8(), R"([0, 10, 15, 20, 25])"); + std::shared_ptr recordBatch = + RecordBatch::Make(*schema, row_count, {string_array, uint8_array}); + ASSERT_OK(prepared_statement->SetParameters(recordBatch)); + }, + &schema, sql_client_); +} + +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/example/sqlite_server.cc b/cpp/src/arrow/flight/sql/example/sqlite_server.cc new file mode 100644 index 00000000000..dde364f64e3 --- /dev/null +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.cc @@ -0,0 +1,813 @@ +// 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. + +#include "arrow/flight/sql/example/sqlite_server.h" + +#include + +#include +#include +#include +#include + +#include "arrow/api.h" +#include "arrow/flight/sql/example/sqlite_sql_info.h" +#include "arrow/flight/sql/example/sqlite_statement.h" +#include "arrow/flight/sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/sql/example/sqlite_tables_schema_batch_reader.h" +#include "arrow/flight/sql/server.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +namespace { + +/// \brief Gets a SqliteStatement by given handle +arrow::Result> GetStatementByHandle( + const std::map>& prepared_statements, + const std::string& handle) { + auto search = prepared_statements.find(handle); + if (search == prepared_statements.end()) { + return Status::Invalid("Prepared statement not found"); + } + + return search->second; +} + +std::string PrepareQueryForGetTables(const GetTables& command) { + std::stringstream table_query; + + table_query << "SELECT null as catalog_name, null as schema_name, name as " + "table_name, type as table_type FROM sqlite_master where 1=1"; + + if (command.catalog.has_value()) { + table_query << " and catalog_name='" << command.catalog.value() << "'"; + } + + if (command.db_schema_filter_pattern.has_value()) { + table_query << " and schema_name LIKE '" << command.db_schema_filter_pattern.value() + << "'"; + } + + if (command.table_name_filter_pattern.has_value()) { + table_query << " and table_name LIKE '" << command.table_name_filter_pattern.value() + << "'"; + } + + if (!command.table_types.empty()) { + table_query << " and table_type IN ("; + size_t size = command.table_types.size(); + for (size_t i = 0; i < size; i++) { + table_query << "'" << command.table_types[i] << "'"; + if (size - 1 != i) { + table_query << ","; + } + } + + table_query << ")"; + } + + table_query << " order by table_name"; + return table_query.str(); +} + +Status SetParametersOnSQLiteStatement(sqlite3_stmt* stmt, FlightMessageReader* reader) { + FlightStreamChunk chunk; + while (true) { + RETURN_NOT_OK(reader->Next(&chunk)); + std::shared_ptr& record_batch = chunk.data; + if (record_batch == nullptr) break; + + const int64_t num_rows = record_batch->num_rows(); + const int& num_columns = record_batch->num_columns(); + + for (int i = 0; i < num_rows; ++i) { + for (int c = 0; c < num_columns; ++c) { + const std::shared_ptr& column = record_batch->column(c); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr scalar, column->GetScalar(i)); + + auto& holder = static_cast(*scalar).value; + + switch (holder->type->id()) { + case Type::INT64: { + int64_t value = static_cast(*holder).value; + sqlite3_bind_int64(stmt, c + 1, value); + break; + } + case Type::FLOAT: { + double value = static_cast(*holder).value; + sqlite3_bind_double(stmt, c + 1, value); + break; + } + case Type::STRING: { + std::shared_ptr buffer = static_cast(*holder).value; + sqlite3_bind_text(stmt, c + 1, reinterpret_cast(buffer->data()), + static_cast(buffer->size()), SQLITE_TRANSIENT); + break; + } + case Type::BINARY: { + std::shared_ptr buffer = static_cast(*holder).value; + sqlite3_bind_blob(stmt, c + 1, buffer->data(), + static_cast(buffer->size()), SQLITE_TRANSIENT); + break; + } + default: + return Status::Invalid("Received unsupported data type: ", + holder->type->ToString()); + } + } + } + } + + return Status::OK(); +} + +arrow::Result> DoGetSQLiteQuery( + sqlite3* db, const std::string& query, const std::shared_ptr& schema) { + std::shared_ptr statement; + + ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db, query)); + + std::shared_ptr reader; + ARROW_ASSIGN_OR_RAISE(reader, SqliteStatementBatchReader::Create(statement, schema)); + + return std::unique_ptr(new RecordBatchStream(reader)); +} + +arrow::Result> GetFlightInfoForCommand( + const FlightDescriptor& descriptor, const std::shared_ptr& schema) { + std::vector endpoints{FlightEndpoint{{descriptor.cmd}, {}}}; + ARROW_ASSIGN_OR_RAISE(auto result, + FlightInfo::Make(*schema, descriptor, endpoints, -1, -1)) + + return std::unique_ptr(new FlightInfo(result)); +} + +std::string PrepareQueryForGetImportedOrExportedKeys(const std::string& filter) { + return R"(SELECT * FROM (SELECT NULL AS pk_catalog_name, + NULL AS pk_schema_name, + p."table" AS pk_table_name, + p."to" AS pk_column_name, + NULL AS fk_catalog_name, + NULL AS fk_schema_name, + m.name AS fk_table_name, + p."from" AS fk_column_name, + p.seq AS key_sequence, + NULL AS pk_key_name, + NULL AS fk_key_name, + CASE + WHEN p.on_update = 'CASCADE' THEN 0 + WHEN p.on_update = 'RESTRICT' THEN 1 + WHEN p.on_update = 'SET NULL' THEN 2 + WHEN p.on_update = 'NO ACTION' THEN 3 + WHEN p.on_update = 'SET DEFAULT' THEN 4 + END AS update_rule, + CASE + WHEN p.on_delete = 'CASCADE' THEN 0 + WHEN p.on_delete = 'RESTRICT' THEN 1 + WHEN p.on_delete = 'SET NULL' THEN 2 + WHEN p.on_delete = 'NO ACTION' THEN 3 + WHEN p.on_delete = 'SET DEFAULT' THEN 4 + END AS delete_rule + FROM sqlite_master m + JOIN pragma_foreign_key_list(m.name) p ON m.name != p."table" + WHERE m.type = 'table') WHERE )" + + filter + R"( ORDER BY + pk_catalog_name, pk_schema_name, pk_table_name, pk_key_name, key_sequence)"; +} + +} // namespace + +std::shared_ptr GetArrowType(const char* sqlite_type) { + if (sqlite_type == NULLPTR) { + // SQLite may not know the column type yet. + return null(); + } + + if (boost::iequals(sqlite_type, "int") || boost::iequals(sqlite_type, "integer")) { + return int64(); + } else if (boost::iequals(sqlite_type, "REAL")) { + return float64(); + } else if (boost::iequals(sqlite_type, "BLOB")) { + return binary(); + } else if (boost::iequals(sqlite_type, "TEXT") || + boost::istarts_with(sqlite_type, "char") || + boost::istarts_with(sqlite_type, "varchar")) { + return utf8(); + } else { + throw std::invalid_argument("Invalid SQLite type: " + std::string(sqlite_type)); + } +} + +class SQLiteFlightSqlServer::Impl { + sqlite3* db_; + std::map> prepared_statements_; + std::default_random_engine gen_; + + public: + explicit Impl(sqlite3* db) : db_(db) {} + + ~Impl() { sqlite3_close(db_); } + + std::string GenerateRandomString() { + uint32_t length = 16; + + std::uniform_int_distribution dist('0', 'z'); + std::string ret(length, 0); + auto get_random_char = [&]() { return dist(gen_); }; + std::generate_n(ret.begin(), length, get_random_char); + return ret; + } + + arrow::Result> GetFlightInfoStatement( + const ServerCallContext& context, const StatementQuery& command, + const FlightDescriptor& descriptor) { + const std::string& query = command.query; + + ARROW_ASSIGN_OR_RAISE(auto statement, SqliteStatement::Create(db_, query)); + + ARROW_ASSIGN_OR_RAISE(auto schema, statement->GetSchema()); + + ARROW_ASSIGN_OR_RAISE(auto ticket_string, CreateStatementQueryTicket(query)); + std::vector endpoints{FlightEndpoint{{ticket_string}, {}}}; + ARROW_ASSIGN_OR_RAISE(auto result, + FlightInfo::Make(*schema, descriptor, endpoints, -1, -1)) + + return std::unique_ptr(new FlightInfo(result)); + } + + arrow::Result> DoGetStatement( + const ServerCallContext& context, const StatementQueryTicket& command) { + const std::string& sql = command.statement_handle; + + std::shared_ptr statement; + ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, sql)); + + std::shared_ptr reader; + ARROW_ASSIGN_OR_RAISE(reader, SqliteStatementBatchReader::Create(statement)); + + return std::unique_ptr(new RecordBatchStream(reader)); + } + + arrow::Result> GetFlightInfoCatalogs( + const ServerCallContext& context, const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetCatalogsSchema()); + } + + arrow::Result> DoGetCatalogs( + const ServerCallContext& context) { + // As SQLite doesn't support catalogs, this will return an empty record batch. + + const std::shared_ptr& schema = SqlSchema::GetCatalogsSchema(); + + StringBuilder catalog_name_builder; + ARROW_ASSIGN_OR_RAISE(auto catalog_name, catalog_name_builder.Finish()); + + const std::shared_ptr& batch = + RecordBatch::Make(schema, 0, {catalog_name}); + + ARROW_ASSIGN_OR_RAISE(auto reader, RecordBatchReader::Make({batch})); + + return std::unique_ptr(new RecordBatchStream(reader)); + } + + arrow::Result> GetFlightInfoSchemas( + const ServerCallContext& context, const GetDbSchemas& command, + const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetDbSchemasSchema()); + } + + arrow::Result> DoGetDbSchemas( + const ServerCallContext& context, const GetDbSchemas& command) { + // As SQLite doesn't support schemas, this will return an empty record batch. + + const std::shared_ptr& schema = SqlSchema::GetDbSchemasSchema(); + + StringBuilder catalog_name_builder; + ARROW_ASSIGN_OR_RAISE(auto catalog_name, catalog_name_builder.Finish()); + StringBuilder schema_name_builder; + ARROW_ASSIGN_OR_RAISE(auto schema_name, schema_name_builder.Finish()); + + const std::shared_ptr& batch = + RecordBatch::Make(schema, 0, {catalog_name, schema_name}); + + ARROW_ASSIGN_OR_RAISE(auto reader, RecordBatchReader::Make({batch})); + + return std::unique_ptr(new RecordBatchStream(reader)); + } + + arrow::Result> GetFlightInfoTables( + const ServerCallContext& context, const GetTables& command, + const FlightDescriptor& descriptor) { + std::vector endpoints{FlightEndpoint{{descriptor.cmd}, {}}}; + + bool include_schema = command.include_schema; + + ARROW_ASSIGN_OR_RAISE( + auto result, + FlightInfo::Make(include_schema ? *SqlSchema::GetTablesSchemaWithIncludedSchema() + : *SqlSchema::GetTablesSchema(), + descriptor, endpoints, -1, -1)) + + return std::unique_ptr(new FlightInfo(result)); + } + + arrow::Result> DoGetTables( + const ServerCallContext& context, const GetTables& command) { + std::string query = PrepareQueryForGetTables(command); + + std::shared_ptr statement; + ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, query)); + + std::shared_ptr reader; + ARROW_ASSIGN_OR_RAISE(reader, SqliteStatementBatchReader::Create( + statement, SqlSchema::GetTablesSchema())); + + if (command.include_schema) { + std::shared_ptr table_schema_reader = + std::make_shared(reader, query, db_); + return std::unique_ptr( + new RecordBatchStream(table_schema_reader)); + } else { + return std::unique_ptr(new RecordBatchStream(reader)); + } + } + + arrow::Result DoPutCommandStatementUpdate(const ServerCallContext& context, + const StatementUpdate& command) { + const std::string& sql = command.query; + + ARROW_ASSIGN_OR_RAISE(auto statement, SqliteStatement::Create(db_, sql)); + + return statement->ExecuteUpdate(); + } + + arrow::Result CreatePreparedStatement( + const ServerCallContext& context, + const ActionCreatePreparedStatementRequest& request) { + std::shared_ptr statement; + ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, request.query)); + const std::string handle = GenerateRandomString(); + prepared_statements_[handle] = statement; + + ARROW_ASSIGN_OR_RAISE(auto dataset_schema, statement->GetSchema()); + + sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); + const int parameter_count = sqlite3_bind_parameter_count(stmt); + std::vector> parameter_fields; + parameter_fields.reserve(parameter_count); + + // As SQLite doesn't know the parameter types before executing the query, the + // example server is accepting any SQLite supported type as input by using a dense + // union. + const std::shared_ptr& dense_union_type = GetUnknownColumnDataType(); + + for (int i = 0; i < parameter_count; i++) { + const char* parameter_name_chars = sqlite3_bind_parameter_name(stmt, i + 1); + std::string parameter_name; + if (parameter_name_chars == NULLPTR) { + parameter_name = std::string("parameter_") + std::to_string(i + 1); + } else { + parameter_name = parameter_name_chars; + } + parameter_fields.push_back(field(parameter_name, dense_union_type)); + } + + const std::shared_ptr& parameter_schema = arrow::schema(parameter_fields); + + ActionCreatePreparedStatementResult result{.dataset_schema = dataset_schema, + .parameter_schema = parameter_schema, + .prepared_statement_handle = handle}; + + return result; + } + + Status ClosePreparedStatement(const ServerCallContext& context, + const ActionClosePreparedStatementRequest& request) { + const std::string& prepared_statement_handle = request.prepared_statement_handle; + + auto search = prepared_statements_.find(prepared_statement_handle); + if (search != prepared_statements_.end()) { + prepared_statements_.erase(prepared_statement_handle); + } else { + return Status::Invalid("Prepared statement not found"); + } + + return Status::OK(); + } + + arrow::Result> GetFlightInfoPreparedStatement( + const ServerCallContext& context, const PreparedStatementQuery& command, + const FlightDescriptor& descriptor) { + const std::string& prepared_statement_handle = command.prepared_statement_handle; + + auto search = prepared_statements_.find(prepared_statement_handle); + if (search == prepared_statements_.end()) { + return Status::Invalid("Prepared statement not found"); + } + + std::shared_ptr statement = search->second; + + ARROW_ASSIGN_OR_RAISE(auto schema, statement->GetSchema()); + + return GetFlightInfoForCommand(descriptor, schema); + } + + arrow::Result> DoGetPreparedStatement( + const ServerCallContext& context, const PreparedStatementQuery& command) { + const std::string& prepared_statement_handle = command.prepared_statement_handle; + + auto search = prepared_statements_.find(prepared_statement_handle); + if (search == prepared_statements_.end()) { + return Status::Invalid("Prepared statement not found"); + } + + std::shared_ptr statement = search->second; + + std::shared_ptr reader; + ARROW_ASSIGN_OR_RAISE(reader, SqliteStatementBatchReader::Create(statement)); + + return std::unique_ptr(new RecordBatchStream(reader)); + } + + Status DoPutPreparedStatementQuery(const ServerCallContext& context, + const PreparedStatementQuery& command, + FlightMessageReader* reader, + FlightMetadataWriter* writer) { + const std::string& prepared_statement_handle = command.prepared_statement_handle; + ARROW_ASSIGN_OR_RAISE( + auto statement, + GetStatementByHandle(prepared_statements_, prepared_statement_handle)); + + sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); + ARROW_RETURN_NOT_OK(SetParametersOnSQLiteStatement(stmt, reader)); + + return Status::OK(); + } + + arrow::Result DoPutPreparedStatementUpdate( + const ServerCallContext& context, const PreparedStatementUpdate& command, + FlightMessageReader* reader) { + const std::string& prepared_statement_handle = command.prepared_statement_handle; + ARROW_ASSIGN_OR_RAISE( + auto statement, + GetStatementByHandle(prepared_statements_, prepared_statement_handle)); + + sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); + ARROW_RETURN_NOT_OK(SetParametersOnSQLiteStatement(stmt, reader)); + + return statement->ExecuteUpdate(); + } + + arrow::Result> GetFlightInfoTableTypes( + const ServerCallContext& context, const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetTableTypesSchema()); + } + + arrow::Result> DoGetTableTypes( + const ServerCallContext& context) { + std::string query = "SELECT DISTINCT type as table_type FROM sqlite_master"; + + return DoGetSQLiteQuery(db_, query, SqlSchema::GetTableTypesSchema()); + } + + arrow::Result> GetFlightInfoPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command, + const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetPrimaryKeysSchema()); + } + + arrow::Result> DoGetPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command) { + std::stringstream table_query; + + // The field key_name can not be recovered by the sqlite, so it is being set + // to null following the same pattern for catalog_name and schema_name. + table_query << "SELECT null as catalog_name, null as schema_name, table_name, " + "name as column_name, pk as key_sequence, null as key_name\n" + "FROM pragma_table_info(table_name)\n" + " JOIN (SELECT null as catalog_name, null as schema_name, name as " + "table_name, type as table_type\n" + "FROM sqlite_master) where 1=1 and pk != 0"; + + const TableRef& table_ref = command.table_ref; + if (table_ref.catalog.has_value()) { + table_query << " and catalog_name LIKE '" << table_ref.catalog.value() << "'"; + } + + if (table_ref.db_schema.has_value()) { + table_query << " and schema_name LIKE '" << table_ref.db_schema.value() << "'"; + } + + table_query << " and table_name LIKE '" << table_ref.table << "'"; + + return DoGetSQLiteQuery(db_, table_query.str(), SqlSchema::GetPrimaryKeysSchema()); + } + + arrow::Result> GetFlightInfoImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command, + const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetImportedKeysSchema()); + } + + arrow::Result> DoGetImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command) { + const TableRef& table_ref = command.table_ref; + std::string filter = "fk_table_name = '" + table_ref.table + "'"; + if (table_ref.catalog.has_value()) { + filter += " AND fk_catalog_name = '" + table_ref.catalog.value() + "'"; + } + if (table_ref.db_schema.has_value()) { + filter += " AND fk_schema_name = '" + table_ref.db_schema.value() + "'"; + } + std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); + + return DoGetSQLiteQuery(db_, query, SqlSchema::GetImportedKeysSchema()); + } + + arrow::Result> GetFlightInfoExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command, + const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetExportedKeysSchema()); + } + + arrow::Result> DoGetExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command) { + const TableRef& table_ref = command.table_ref; + std::string filter = "pk_table_name = '" + table_ref.table + "'"; + if (table_ref.catalog.has_value()) { + filter += " AND pk_catalog_name = '" + table_ref.catalog.value() + "'"; + } + if (table_ref.db_schema.has_value()) { + filter += " AND pk_schema_name = '" + table_ref.db_schema.value() + "'"; + } + std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); + + return DoGetSQLiteQuery(db_, query, SqlSchema::GetExportedKeysSchema()); + } + + arrow::Result> GetFlightInfoCrossReference( + const ServerCallContext& context, const GetCrossReference& command, + const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetCrossReferenceSchema()); + } + + arrow::Result> DoGetCrossReference( + const ServerCallContext& context, const GetCrossReference& command) { + const TableRef& pk_table_ref = command.pk_table_ref; + std::string filter = "pk_table_name = '" + pk_table_ref.table + "'"; + if (pk_table_ref.catalog.has_value()) { + filter += " AND pk_catalog_name = '" + pk_table_ref.catalog.value() + "'"; + } + if (pk_table_ref.db_schema.has_value()) { + filter += " AND pk_schema_name = '" + pk_table_ref.db_schema.value() + "'"; + } + + const TableRef& fk_table_ref = command.fk_table_ref; + filter += " AND fk_table_name = '" + fk_table_ref.table + "'"; + if (fk_table_ref.catalog.has_value()) { + filter += " AND fk_catalog_name = '" + fk_table_ref.catalog.value() + "'"; + } + if (fk_table_ref.db_schema.has_value()) { + filter += " AND fk_schema_name = '" + fk_table_ref.db_schema.value() + "'"; + } + std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); + + return DoGetSQLiteQuery(db_, query, SqlSchema::GetCrossReferenceSchema()); + } + + Status ExecuteSql(const std::string& sql) { + char* err_msg = nullptr; + int rc = sqlite3_exec(db_, sql.c_str(), nullptr, nullptr, &err_msg); + if (rc != SQLITE_OK) { + std::string error_msg; + if (err_msg != nullptr) { + error_msg = err_msg; + } + sqlite3_free(err_msg); + return Status::ExecutionError(error_msg); + } + return Status::OK(); + } +}; + +SQLiteFlightSqlServer::SQLiteFlightSqlServer(std::shared_ptr impl) + : impl_(std::move(impl)) {} + +arrow::Result> SQLiteFlightSqlServer::Create() { + sqlite3* db = nullptr; + + if (sqlite3_open(":memory:", &db)) { + std::string err_msg = "Can't open database: "; + if (db != nullptr) { + err_msg += sqlite3_errmsg(db); + sqlite3_close(db); + } else { + err_msg += "Unable to start SQLite. Insufficient memory"; + } + + return Status::Invalid(err_msg); + } + + std::shared_ptr impl = std::make_shared(db); + + std::shared_ptr result(new SQLiteFlightSqlServer(impl)); + for (const auto& id_to_result : GetSqlInfoResultMap()) { + result->RegisterSqlInfo(id_to_result.first, id_to_result.second); + } + + ARROW_RETURN_NOT_OK(result->ExecuteSql(R"( + CREATE TABLE foreignTable ( + id INTEGER PRIMARY KEY AUTOINCREMENT, + foreignName varchar(100), + value int); + + CREATE TABLE intTable ( + id INTEGER PRIMARY KEY AUTOINCREMENT, + keyName varchar(100), + value int, + foreignId int references foreignTable(id)); + + INSERT INTO foreignTable (foreignName, value) VALUES ('keyOne', 1); + INSERT INTO foreignTable (foreignName, value) VALUES ('keyTwo', 0); + INSERT INTO foreignTable (foreignName, value) VALUES ('keyThree', -1); + INSERT INTO intTable (keyName, value, foreignId) VALUES ('one', 1, 1); + INSERT INTO intTable (keyName, value, foreignId) VALUES ('zero', 0, 1); + INSERT INTO intTable (keyName, value, foreignId) VALUES ('negative one', -1, 1); + INSERT INTO intTable (keyName, value, foreignId) VALUES (NULL, NULL, NULL); + )")); + + return result; +} + +SQLiteFlightSqlServer::~SQLiteFlightSqlServer() = default; + +Status SQLiteFlightSqlServer::ExecuteSql(const std::string& sql) { + return impl_->ExecuteSql(sql); +} + +arrow::Result> SQLiteFlightSqlServer::GetFlightInfoStatement( + const ServerCallContext& context, const StatementQuery& command, + const FlightDescriptor& descriptor) { + return impl_->GetFlightInfoStatement(context, command, descriptor); +} + +arrow::Result> SQLiteFlightSqlServer::DoGetStatement( + const ServerCallContext& context, const StatementQueryTicket& command) { + return impl_->DoGetStatement(context, command); +} + +arrow::Result> SQLiteFlightSqlServer::GetFlightInfoCatalogs( + const ServerCallContext& context, const FlightDescriptor& descriptor) { + return impl_->GetFlightInfoCatalogs(context, descriptor); +} + +arrow::Result> SQLiteFlightSqlServer::DoGetCatalogs( + const ServerCallContext& context) { + return impl_->DoGetCatalogs(context); +} + +arrow::Result> SQLiteFlightSqlServer::GetFlightInfoSchemas( + const ServerCallContext& context, const GetDbSchemas& command, + const FlightDescriptor& descriptor) { + return impl_->GetFlightInfoSchemas(context, command, descriptor); +} + +arrow::Result> SQLiteFlightSqlServer::DoGetDbSchemas( + const ServerCallContext& context, const GetDbSchemas& command) { + return impl_->DoGetDbSchemas(context, command); +} + +arrow::Result> SQLiteFlightSqlServer::GetFlightInfoTables( + const ServerCallContext& context, const GetTables& command, + const FlightDescriptor& descriptor) { + return impl_->GetFlightInfoTables(context, command, descriptor); +} + +arrow::Result> SQLiteFlightSqlServer::DoGetTables( + const ServerCallContext& context, const GetTables& command) { + return impl_->DoGetTables(context, command); +} + +arrow::Result SQLiteFlightSqlServer::DoPutCommandStatementUpdate( + const ServerCallContext& context, const StatementUpdate& command) { + return impl_->DoPutCommandStatementUpdate(context, command); +} + +arrow::Result +SQLiteFlightSqlServer::CreatePreparedStatement( + const ServerCallContext& context, + const ActionCreatePreparedStatementRequest& request) { + return impl_->CreatePreparedStatement(context, request); +} + +Status SQLiteFlightSqlServer::ClosePreparedStatement( + const ServerCallContext& context, + const ActionClosePreparedStatementRequest& request) { + return impl_->ClosePreparedStatement(context, request); +} + +arrow::Result> +SQLiteFlightSqlServer::GetFlightInfoPreparedStatement( + const ServerCallContext& context, const PreparedStatementQuery& command, + const FlightDescriptor& descriptor) { + return impl_->GetFlightInfoPreparedStatement(context, command, descriptor); +} + +arrow::Result> +SQLiteFlightSqlServer::DoGetPreparedStatement(const ServerCallContext& context, + const PreparedStatementQuery& command) { + return impl_->DoGetPreparedStatement(context, command); +} + +Status SQLiteFlightSqlServer::DoPutPreparedStatementQuery( + const ServerCallContext& context, const PreparedStatementQuery& command, + FlightMessageReader* reader, FlightMetadataWriter* writer) { + return impl_->DoPutPreparedStatementQuery(context, command, reader, writer); +} + +arrow::Result SQLiteFlightSqlServer::DoPutPreparedStatementUpdate( + const ServerCallContext& context, const PreparedStatementUpdate& command, + FlightMessageReader* reader) { + return impl_->DoPutPreparedStatementUpdate(context, command, reader); +} + +arrow::Result> SQLiteFlightSqlServer::GetFlightInfoTableTypes( + const ServerCallContext& context, const FlightDescriptor& descriptor) { + return impl_->GetFlightInfoTableTypes(context, descriptor); +} + +arrow::Result> SQLiteFlightSqlServer::DoGetTableTypes( + const ServerCallContext& context) { + return impl_->DoGetTableTypes(context); +} + +arrow::Result> +SQLiteFlightSqlServer::GetFlightInfoPrimaryKeys(const ServerCallContext& context, + const GetPrimaryKeys& command, + const FlightDescriptor& descriptor) { + return impl_->GetFlightInfoPrimaryKeys(context, command, descriptor); +} + +arrow::Result> SQLiteFlightSqlServer::DoGetPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command) { + return impl_->DoGetPrimaryKeys(context, command); +} + +arrow::Result> +SQLiteFlightSqlServer::GetFlightInfoImportedKeys(const ServerCallContext& context, + const GetImportedKeys& command, + const FlightDescriptor& descriptor) { + return impl_->GetFlightInfoImportedKeys(context, command, descriptor); +} + +arrow::Result> SQLiteFlightSqlServer::DoGetImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command) { + return impl_->DoGetImportedKeys(context, command); +} + +arrow::Result> +SQLiteFlightSqlServer::GetFlightInfoExportedKeys(const ServerCallContext& context, + const GetExportedKeys& command, + const FlightDescriptor& descriptor) { + return impl_->GetFlightInfoExportedKeys(context, command, descriptor); +} + +arrow::Result> SQLiteFlightSqlServer::DoGetExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command) { + return impl_->DoGetExportedKeys(context, command); +} + +arrow::Result> +SQLiteFlightSqlServer::GetFlightInfoCrossReference(const ServerCallContext& context, + const GetCrossReference& command, + const FlightDescriptor& descriptor) { + return impl_->GetFlightInfoCrossReference(context, command, descriptor); +} + +arrow::Result> +SQLiteFlightSqlServer::DoGetCrossReference(const ServerCallContext& context, + const GetCrossReference& command) { + return impl_->DoGetCrossReference(context, command); +} + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/example/sqlite_server.h b/cpp/src/arrow/flight/sql/example/sqlite_server.h new file mode 100644 index 00000000000..b2954b8703e --- /dev/null +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.h @@ -0,0 +1,142 @@ +// 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. + +#pragma once + +#include + +#include +#include + +#include "arrow/api.h" +#include "arrow/flight/sql/example/sqlite_statement.h" +#include "arrow/flight/sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/sql/server.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +/// \brief Convert a column type to a ArrowType. +/// \param sqlite_type the sqlite type. +/// \return The equivalent ArrowType. +std::shared_ptr GetArrowType(const char* sqlite_type); + +/// \brief Get the DataType used when parameter type is not known. +/// \return DataType used when parameter type is not known. +inline std::shared_ptr GetUnknownColumnDataType() { + return dense_union({ + field("string", utf8()), + field("bytes", binary()), + field("bigint", int64()), + field("double", float64()), + }); +} + +/// \brief Example implementation of FlightSqlServerBase backed by an in-memory SQLite3 +/// database. +class SQLiteFlightSqlServer : public FlightSqlServerBase { + public: + ~SQLiteFlightSqlServer() override; + + static arrow::Result> Create(); + + /// \brief Auxiliary method used to execute an arbitrary SQL statement on the underlying + /// SQLite database. + Status ExecuteSql(const std::string& sql); + + arrow::Result> GetFlightInfoStatement( + const ServerCallContext& context, const StatementQuery& command, + const FlightDescriptor& descriptor) override; + + arrow::Result> DoGetStatement( + const ServerCallContext& context, const StatementQueryTicket& command) override; + arrow::Result> GetFlightInfoCatalogs( + const ServerCallContext& context, const FlightDescriptor& descriptor) override; + arrow::Result> DoGetCatalogs( + const ServerCallContext& context) override; + arrow::Result> GetFlightInfoSchemas( + const ServerCallContext& context, const GetDbSchemas& command, + const FlightDescriptor& descriptor) override; + arrow::Result> DoGetDbSchemas( + const ServerCallContext& context, const GetDbSchemas& command) override; + arrow::Result DoPutCommandStatementUpdate( + const ServerCallContext& context, const StatementUpdate& update) override; + arrow::Result CreatePreparedStatement( + const ServerCallContext& context, + const ActionCreatePreparedStatementRequest& request) override; + Status ClosePreparedStatement( + const ServerCallContext& context, + const ActionClosePreparedStatementRequest& request) override; + arrow::Result> GetFlightInfoPreparedStatement( + const ServerCallContext& context, const PreparedStatementQuery& command, + const FlightDescriptor& descriptor) override; + arrow::Result> DoGetPreparedStatement( + const ServerCallContext& context, const PreparedStatementQuery& command) override; + Status DoPutPreparedStatementQuery(const ServerCallContext& context, + const PreparedStatementQuery& command, + FlightMessageReader* reader, + FlightMetadataWriter* writer) override; + arrow::Result DoPutPreparedStatementUpdate( + const ServerCallContext& context, const PreparedStatementUpdate& command, + FlightMessageReader* reader) override; + + arrow::Result> GetFlightInfoTables( + const ServerCallContext& context, const GetTables& command, + const FlightDescriptor& descriptor) override; + + arrow::Result> DoGetTables( + const ServerCallContext& context, const GetTables& command) override; + arrow::Result> GetFlightInfoTableTypes( + const ServerCallContext& context, const FlightDescriptor& descriptor) override; + arrow::Result> DoGetTableTypes( + const ServerCallContext& context) override; + arrow::Result> GetFlightInfoImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command, + const FlightDescriptor& descriptor) override; + arrow::Result> DoGetImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command) override; + arrow::Result> GetFlightInfoExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command, + const FlightDescriptor& descriptor) override; + arrow::Result> DoGetExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command) override; + arrow::Result> GetFlightInfoCrossReference( + const ServerCallContext& context, const GetCrossReference& command, + const FlightDescriptor& descriptor) override; + arrow::Result> DoGetCrossReference( + const ServerCallContext& context, const GetCrossReference& command) override; + + arrow::Result> GetFlightInfoPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command, + const FlightDescriptor& descriptor) override; + + arrow::Result> DoGetPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command) override; + + private: + class Impl; + std::shared_ptr impl_; + + explicit SQLiteFlightSqlServer(std::shared_ptr impl); +}; + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc new file mode 100644 index 00000000000..94f25b39017 --- /dev/null +++ b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc @@ -0,0 +1,223 @@ +// 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. + +#include "arrow/flight/sql/example/sqlite_sql_info.h" + +#include "arrow/flight/sql/types.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +/// \brief Gets the mapping from SQL info ids to SqlInfoResult instances. +/// \return the cache. +SqlInfoResultMap GetSqlInfoResultMap() { + return { + {SqlInfoOptions::SqlInfo::FLIGHT_SQL_SERVER_NAME, + SqlInfoResult(std::string("db_name"))}, + {SqlInfoOptions::SqlInfo::FLIGHT_SQL_SERVER_VERSION, + SqlInfoResult(std::string("sqlite 3"))}, + {SqlInfoOptions::SqlInfo::FLIGHT_SQL_SERVER_ARROW_VERSION, + SqlInfoResult(std::string("7.0.0-SNAPSHOT" /* Only an example */))}, + {SqlInfoOptions::SqlInfo::FLIGHT_SQL_SERVER_READ_ONLY, SqlInfoResult(false)}, + {SqlInfoOptions::SqlInfo::SQL_DDL_CATALOG, + SqlInfoResult(false /* SQLite 3 does not support catalogs */)}, + {SqlInfoOptions::SqlInfo::SQL_DDL_SCHEMA, + SqlInfoResult(false /* SQLite 3 does not support schemas */)}, + {SqlInfoOptions::SqlInfo::SQL_DDL_TABLE, SqlInfoResult(true)}, + {SqlInfoOptions::SqlInfo::SQL_IDENTIFIER_CASE, + SqlInfoResult(int64_t(SqlInfoOptions::SqlSupportedCaseSensitivity:: + SQL_CASE_SENSITIVITY_CASE_INSENSITIVE))}, + {SqlInfoOptions::SqlInfo::SQL_IDENTIFIER_QUOTE_CHAR, + SqlInfoResult(std::string("\""))}, + {SqlInfoOptions::SqlInfo::SQL_QUOTED_IDENTIFIER_CASE, + SqlInfoResult(int64_t(SqlInfoOptions::SqlSupportedCaseSensitivity:: + SQL_CASE_SENSITIVITY_CASE_INSENSITIVE))}, + {SqlInfoOptions::SqlInfo::SQL_ALL_TABLES_ARE_SELECTABLE, SqlInfoResult(true)}, + {SqlInfoOptions::SqlInfo::SQL_NULL_ORDERING, + SqlInfoResult( + int64_t(SqlInfoOptions::SqlNullOrdering::SQL_NULLS_SORTED_AT_START))}, + {SqlInfoOptions::SqlInfo::SQL_KEYWORDS, + SqlInfoResult(std::vector({"ABORT", + "ACTION", + "ADD", + "AFTER", + "ALL", + "ALTER", + "ALWAYS", + "ANALYZE", + "AND", + "AS", + "ASC", + "ATTACH", + "AUTOINCREMENT", + "BEFORE", + "BEGIN", + "BETWEEN", + "BY", + "CASCADE", + "CASE", + "CAST", + "CHECK", + "COLLATE", + "COLUMN", + "COMMIT", + "CONFLICT", + "CONSTRAINT", + "CREATE", + "CROSS", + "CURRENT", + "CURRENT_DATE", + "CURRENT_TIME", + "CURRENT_TIMESTAMP", + "DATABASE", + "DEFAULT", + "DEFERRABLE", + "DEFERRED", + "DELETE", + "DESC", + "DETACH", + "DISTINCT", + "DO", + "DROP", + "EACH", + "ELSE", + "END", + "ESCAPE", + "EXCEPT", + "EXCLUDE", + "EXCLUSIVE", + "EXISTS", + "EXPLAIN", + "FAIL", + "FILTER", + "FIRST", + "FOLLOWING", + "FOR", + "FOREIGN", + "FROM", + "FULL", + "GENERATED", + "GLOB", + "GROUP", + "GROUPS", + "HAVING", + "IF", + "IGNORE", + "IMMEDIATE", + "IN", + "INDEX", + "INDEXED", + "INITIALLY", + "INNER", + "INSERT", + "INSTEAD", + "INTERSECT", + "INTO", + "IS", + "ISNULL", + "JOIN", + "KEY", + "LAST", + "LEFT", + "LIKE", + "LIMIT", + "MATCH", + "MATERIALIZED", + "NATURAL", + "NO", + "NOT", + "NOTHING", + "NOTNULL", + "NULL", + "NULLS", + "OF", + "OFFSET", + "ON", + "OR", + "ORDER", + "OTHERS", + "OUTER", + "OVER", + "PARTITION", + "PLAN", + "PRAGMA", + "PRECEDING", + "PRIMARY", + "QUERY", + "RAISE", + "RANGE", + "RECURSIVE", + "REFERENCES", + "REGEXP", + "REINDEX", + "RELEASE", + "RENAME", + "REPLACE", + "RESTRICT", + "RETURNING", + "RIGHT", + "ROLLBACK", + "ROW", + "ROWS", + "SAVEPOINT", + "SELECT", + "SET", + "TABLE", + "TEMP", + "TEMPORARY", + "THEN", + "TIES", + "TO", + "TRANSACTION", + "TRIGGER", + "UNBOUNDED", + "UNION", + "UNIQUE", + "UPDATE", + "USING", + "VACUUM", + "VALUES", + "VIEW", + "VIRTUAL", + "WHEN", + "WHERE", + "WINDOW", + "WITH", + "WITHOUT"}))}, + {SqlInfoOptions::SqlInfo::SQL_NUMERIC_FUNCTIONS, + SqlInfoResult(std::vector( + {"ACOS", "ACOSH", "ASIN", "ASINH", "ATAN", "ATAN2", "ATANH", "CEIL", + "CEILING", "COS", "COSH", "DEGREES", "EXP", "FLOOR", "LN", "LOG", + "LOG", "LOG10", "LOG2", "MOD", "PI", "POW", "POWER", "RADIANS", + "SIN", "SINH", "SQRT", "TAN", "TANH", "TRUNC"}))}, + {SqlInfoOptions::SqlInfo::SQL_STRING_FUNCTIONS, + SqlInfoResult( + std::vector({"SUBSTR", "TRIM", "LTRIM", "RTRIM", "LENGTH", + "REPLACE", "UPPER", "LOWER", "INSTR"}))}, + {SqlInfoOptions::SqlInfo::SQL_SUPPORTS_CONVERT, + SqlInfoResult(std::unordered_map>( + {{SqlInfoOptions::SqlSupportsConvert::SQL_CONVERT_BIGINT, + std::vector( + {SqlInfoOptions::SqlSupportsConvert::SQL_CONVERT_INTEGER})}}))}}; +} + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/example/sqlite_sql_info.h b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.h new file mode 100644 index 00000000000..3c6dd42135e --- /dev/null +++ b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.h @@ -0,0 +1,34 @@ +// 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. + +#pragma once + +#include "arrow/flight/sql/types.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +/// \brief Gets the mapping from SQL info ids to SqlInfoResult instances. +/// \return the cache. +SqlInfoResultMap GetSqlInfoResultMap(); + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/sql/example/sqlite_statement.cc new file mode 100644 index 00000000000..018f8de37db --- /dev/null +++ b/cpp/src/arrow/flight/sql/example/sqlite_statement.cc @@ -0,0 +1,137 @@ +// 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. + +#include "arrow/flight/sql/example/sqlite_statement.h" + +#include + +#include + +#include "arrow/flight/sql/example/sqlite_server.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +std::shared_ptr GetDataTypeFromSqliteType(const int column_type) { + switch (column_type) { + case SQLITE_INTEGER: + return int64(); + case SQLITE_FLOAT: + return float64(); + case SQLITE_BLOB: + return binary(); + case SQLITE_TEXT: + return utf8(); + case SQLITE_NULL: + default: + return null(); + } +} + +arrow::Result> SqliteStatement::Create( + sqlite3* db, const std::string& sql) { + sqlite3_stmt* stmt = nullptr; + int rc = + sqlite3_prepare_v2(db, sql.c_str(), static_cast(sql.size()), &stmt, NULLPTR); + + if (rc != SQLITE_OK) { + std::string err_msg = "Can't prepare statement: " + std::string(sqlite3_errmsg(db)); + if (stmt != nullptr) { + rc = sqlite3_finalize(stmt); + if (rc != SQLITE_OK) { + err_msg += "; Failed to finalize SQLite statement: "; + err_msg += std::string(sqlite3_errmsg(db)); + } + } + return Status::Invalid(err_msg); + } + + std::shared_ptr result(new SqliteStatement(db, stmt)); + return result; +} + +arrow::Result> SqliteStatement::GetSchema() const { + std::vector> fields; + int column_count = sqlite3_column_count(stmt_); + for (int i = 0; i < column_count; i++) { + const char* column_name = sqlite3_column_name(stmt_, i); + + // SQLite does not always provide column types, especially when the statement has not + // been executed yet. Because of this behaviour this method tries to get the column + // types in two attempts: + // 1. Use sqlite3_column_type(), which return SQLITE_NULL if the statement has not + // been executed yet + // 2. Use sqlite3_column_decltype(), which returns correctly if given column is + // declared in the table. + // Because of this limitation, it is not possible to know the column types for some + // prepared statements, in this case it returns a dense_union type covering any type + // SQLite supports. + const int column_type = sqlite3_column_type(stmt_, i); + std::shared_ptr data_type = GetDataTypeFromSqliteType(column_type); + if (data_type->id() == Type::NA) { + // Try to retrieve column type from sqlite3_column_decltype + const char* column_decltype = sqlite3_column_decltype(stmt_, i); + if (column_decltype != NULLPTR) { + data_type = GetArrowType(column_decltype); + } else { + // If it can not determine the actual column type, return a dense_union type + // covering any type SQLite supports. + data_type = GetUnknownColumnDataType(); + } + } + + fields.push_back(arrow::field(column_name, data_type)); + } + + return arrow::schema(fields); +} + +SqliteStatement::~SqliteStatement() { sqlite3_finalize(stmt_); } + +arrow::Result SqliteStatement::Step() { + int rc = sqlite3_step(stmt_); + if (rc == SQLITE_ERROR) { + return Status::ExecutionError("A SQLite runtime error has occurred: ", + sqlite3_errmsg(db_)); + } + + return rc; +} + +arrow::Result SqliteStatement::Reset() { + int rc = sqlite3_reset(stmt_); + if (rc == SQLITE_ERROR) { + return Status::ExecutionError("A SQLite runtime error has occurred: ", + sqlite3_errmsg(db_)); + } + + return rc; +} + +sqlite3_stmt* SqliteStatement::GetSqlite3Stmt() const { return stmt_; } + +arrow::Result SqliteStatement::ExecuteUpdate() { + ARROW_RETURN_NOT_OK(Step()); + return sqlite3_changes(db_); +} + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/example/sqlite_statement.h b/cpp/src/arrow/flight/sql/example/sqlite_statement.h new file mode 100644 index 00000000000..a3f086abc47 --- /dev/null +++ b/cpp/src/arrow/flight/sql/example/sqlite_statement.h @@ -0,0 +1,73 @@ +// 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. + +#pragma once + +#include + +#include +#include + +#include "arrow/type_fwd.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +class SqliteStatement { + public: + /// \brief Creates a SQLite3 statement. + /// \param[in] db SQLite3 database instance. + /// \param[in] sql SQL statement. + /// \return A SqliteStatement object. + static arrow::Result> Create(sqlite3* db, + const std::string& sql); + + ~SqliteStatement(); + + /// \brief Creates an Arrow Schema based on the results of this statement. + /// \return The resulting Schema. + arrow::Result> GetSchema() const; + + /// \brief Steps on underlying sqlite3_stmt. + /// \return The resulting return code from SQLite. + arrow::Result Step(); + + /// \brief Reset the state of the sqlite3_stmt. + /// \return The resulting return code from SQLite. + arrow::Result Reset(); + + /// \brief Returns the underlying sqlite3_stmt. + /// \return A sqlite statement. + sqlite3_stmt* GetSqlite3Stmt() const; + + /// \brief Executes an UPDATE, INSERT or DELETE statement. + /// \return The number of rows changed by execution. + arrow::Result ExecuteUpdate(); + + private: + sqlite3* db_; + sqlite3_stmt* stmt_; + + SqliteStatement(sqlite3* db, sqlite3_stmt* stmt) : db_(db), stmt_(stmt) {} +}; + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.cc new file mode 100644 index 00000000000..a5824ae255f --- /dev/null +++ b/cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.cc @@ -0,0 +1,189 @@ +// 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. + +#include "arrow/flight/sql/example/sqlite_statement_batch_reader.h" + +#include + +#include "arrow/builder.h" +#include "arrow/flight/sql/example/sqlite_statement.h" + +#define STRING_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ + case TYPE_CLASS##Type::type_id: { \ + int bytes = sqlite3_column_bytes(STMT, COLUMN); \ + const unsigned char* string = sqlite3_column_text(STMT, COLUMN); \ + if (string == nullptr) { \ + ARROW_RETURN_NOT_OK( \ + (reinterpret_cast(builder)).AppendNull()); \ + break; \ + } \ + ARROW_RETURN_NOT_OK( \ + (reinterpret_cast(builder)).Append(string, bytes)); \ + break; \ + } + +#define BINARY_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ + case TYPE_CLASS##Type::type_id: { \ + int bytes = sqlite3_column_bytes(STMT, COLUMN); \ + const void* blob = sqlite3_column_blob(STMT, COLUMN); \ + if (blob == nullptr) { \ + ARROW_RETURN_NOT_OK( \ + (reinterpret_cast(builder)).AppendNull()); \ + break; \ + } \ + ARROW_RETURN_NOT_OK( \ + (reinterpret_cast(builder)).Append((char*)blob, bytes)); \ + break; \ + } + +#define INT_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ + case TYPE_CLASS##Type::type_id: { \ + if (sqlite3_column_type(stmt_, i) == SQLITE_NULL) { \ + ARROW_RETURN_NOT_OK( \ + (reinterpret_cast(builder)).AppendNull()); \ + break; \ + } \ + sqlite3_int64 value = sqlite3_column_int64(STMT, COLUMN); \ + ARROW_RETURN_NOT_OK( \ + (reinterpret_cast(builder)).Append(value)); \ + break; \ + } + +#define FLOAT_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ + case TYPE_CLASS##Type::type_id: { \ + if (sqlite3_column_type(stmt_, i) == SQLITE_NULL) { \ + ARROW_RETURN_NOT_OK( \ + (reinterpret_cast(builder)).AppendNull()); \ + break; \ + } \ + double value = sqlite3_column_double(STMT, COLUMN); \ + ARROW_RETURN_NOT_OK( \ + (reinterpret_cast(builder)).Append(value)); \ + break; \ + } + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +// Batch size for SQLite statement results +static constexpr int kMaxBatchSize = 1024; + +std::shared_ptr SqliteStatementBatchReader::schema() const { return schema_; } + +SqliteStatementBatchReader::SqliteStatementBatchReader( + std::shared_ptr statement, std::shared_ptr schema) + : statement_(std::move(statement)), + schema_(std::move(schema)), + rc_(SQLITE_OK), + already_executed_(false) {} + +Result> SqliteStatementBatchReader::Create( + const std::shared_ptr& statement_) { + ARROW_RETURN_NOT_OK(statement_->Step()); + + ARROW_ASSIGN_OR_RAISE(auto schema, statement_->GetSchema()); + + std::shared_ptr result( + new SqliteStatementBatchReader(statement_, schema)); + + return result; +} + +arrow::Result> +SqliteStatementBatchReader::Create(const std::shared_ptr& statement, + const std::shared_ptr& schema) { + std::shared_ptr result( + new SqliteStatementBatchReader(statement, schema)); + + return result; +} + +Status SqliteStatementBatchReader::ReadNext(std::shared_ptr* out) { + sqlite3_stmt* stmt_ = statement_->GetSqlite3Stmt(); + + const int num_fields = schema_->num_fields(); + std::vector> builders(num_fields); + + for (int i = 0; i < num_fields; i++) { + const std::shared_ptr& field = schema_->field(i); + const std::shared_ptr& field_type = field->type(); + + ARROW_RETURN_NOT_OK(MakeBuilder(default_memory_pool(), field_type, &builders[i])); + } + + if (!already_executed_) { + ARROW_ASSIGN_OR_RAISE(rc_, statement_->Reset()); + ARROW_ASSIGN_OR_RAISE(rc_, statement_->Step()); + already_executed_ = true; + } + + int64_t rows = 0; + while (rows < kMaxBatchSize && rc_ == SQLITE_ROW) { + rows++; + for (int i = 0; i < num_fields; i++) { + const std::shared_ptr& field = schema_->field(i); + const std::shared_ptr& field_type = field->type(); + ArrayBuilder& builder = *builders[i]; + + // NOTE: This is not the optimal way of building Arrow vectors. + // That would be to presize the builders to avoiding several resizing operations + // when appending values and also to build one vector at a time. + switch (field_type->id()) { + INT_BUILDER_CASE(Int64, stmt_, i) + INT_BUILDER_CASE(UInt64, stmt_, i) + INT_BUILDER_CASE(Int32, stmt_, i) + INT_BUILDER_CASE(UInt32, stmt_, i) + INT_BUILDER_CASE(Int16, stmt_, i) + INT_BUILDER_CASE(UInt16, stmt_, i) + INT_BUILDER_CASE(Int8, stmt_, i) + INT_BUILDER_CASE(UInt8, stmt_, i) + FLOAT_BUILDER_CASE(Double, stmt_, i) + FLOAT_BUILDER_CASE(Float, stmt_, i) + FLOAT_BUILDER_CASE(HalfFloat, stmt_, i) + BINARY_BUILDER_CASE(Binary, stmt_, i) + BINARY_BUILDER_CASE(LargeBinary, stmt_, i) + STRING_BUILDER_CASE(String, stmt_, i) + STRING_BUILDER_CASE(LargeString, stmt_, i) + default: + return Status::NotImplemented("Not implemented SQLite data conversion to ", + field_type->name()); + } + } + + ARROW_ASSIGN_OR_RAISE(rc_, statement_->Step()); + } + + if (rows > 0) { + std::vector> arrays(builders.size()); + for (int i = 0; i < num_fields; i++) { + ARROW_RETURN_NOT_OK(builders[i]->Finish(&arrays[i])); + } + + *out = RecordBatch::Make(schema_, rows, arrays); + } else { + *out = NULLPTR; + } + + return Status::OK(); +} + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.h new file mode 100644 index 00000000000..8a6bc6078e7 --- /dev/null +++ b/cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.h @@ -0,0 +1,65 @@ +// 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. + +#pragma once + +#include + +#include + +#include "arrow/flight/sql/example/sqlite_statement.h" +#include "arrow/record_batch.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +class SqliteStatementBatchReader : public RecordBatchReader { + public: + /// \brief Creates a RecordBatchReader backed by a SQLite statement. + /// \param[in] statement SQLite statement to be read. + /// \return A SqliteStatementBatchReader. + static arrow::Result> Create( + const std::shared_ptr& statement); + + /// \brief Creates a RecordBatchReader backed by a SQLite statement. + /// \param[in] statement SQLite statement to be read. + /// \param[in] schema Schema to be used on results. + /// \return A SqliteStatementBatchReader.. + static arrow::Result> Create( + const std::shared_ptr& statement, + const std::shared_ptr& schema); + + std::shared_ptr schema() const override; + + Status ReadNext(std::shared_ptr* out) override; + + private: + std::shared_ptr statement_; + std::shared_ptr schema_; + int rc_; + bool already_executed_; + + SqliteStatementBatchReader(std::shared_ptr statement, + std::shared_ptr schema); +}; + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/example/sqlite_tables_schema_batch_reader.cc b/cpp/src/arrow/flight/sql/example/sqlite_tables_schema_batch_reader.cc new file mode 100644 index 00000000000..7fb68a709f8 --- /dev/null +++ b/cpp/src/arrow/flight/sql/example/sqlite_tables_schema_batch_reader.cc @@ -0,0 +1,106 @@ +// 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. + +#include "arrow/flight/sql/example/sqlite_tables_schema_batch_reader.h" + +#include + +#include + +#include "arrow/flight/sql/example/sqlite_server.h" +#include "arrow/flight/sql/example/sqlite_statement.h" +#include "arrow/flight/sql/server.h" +#include "arrow/ipc/writer.h" +#include "arrow/record_batch.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +std::shared_ptr SqliteTablesWithSchemaBatchReader::schema() const { + return SqlSchema::GetTablesSchemaWithIncludedSchema(); +} + +Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* batch) { + std::stringstream schema_query; + + schema_query + << "SELECT table_name, name, type, [notnull] FROM pragma_table_info(table_name)" + << "JOIN(" << main_query_ << ") order by table_name"; + + std::shared_ptr schema_statement; + ARROW_ASSIGN_OR_RAISE(schema_statement, + example::SqliteStatement::Create(db_, schema_query.str())) + + std::shared_ptr first_batch; + + ARROW_RETURN_NOT_OK(reader_->ReadNext(&first_batch)); + + if (!first_batch) { + *batch = NULLPTR; + return Status::OK(); + } + + const std::shared_ptr table_name_array = + first_batch->GetColumnByName("table_name"); + + BinaryBuilder schema_builder; + + auto* string_array = reinterpret_cast(table_name_array.get()); + + std::vector> column_fields; + for (int i = 0; i < table_name_array->length(); i++) { + const std::string& table_name = string_array->GetString(i); + + while (sqlite3_step(schema_statement->GetSqlite3Stmt()) == SQLITE_ROW) { + std::string sqlite_table_name = std::string(reinterpret_cast( + sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 0))); + if (sqlite_table_name == table_name) { + const char* column_name = reinterpret_cast( + sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 1)); + const char* column_type = reinterpret_cast( + sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 2)); + int nullable = sqlite3_column_int(schema_statement->GetSqlite3Stmt(), 3); + + column_fields.push_back( + arrow::field(column_name, GetArrowType(column_type), nullable == 0, NULL)); + } + } + const arrow::Result>& value = + ipc::SerializeSchema(*arrow::schema(column_fields)); + + std::shared_ptr schema_buffer; + ARROW_ASSIGN_OR_RAISE(schema_buffer, value); + + column_fields.clear(); + ARROW_RETURN_NOT_OK( + schema_builder.Append(schema_buffer->data(), schema_buffer->size())); + } + + std::shared_ptr schema_array; + ARROW_RETURN_NOT_OK(schema_builder.Finish(&schema_array)); + + ARROW_ASSIGN_OR_RAISE(*batch, first_batch->AddColumn(4, "table_schema", schema_array)); + + return Status::OK(); +} + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/sql/example/sqlite_tables_schema_batch_reader.h new file mode 100644 index 00000000000..ecba88efb2f --- /dev/null +++ b/cpp/src/arrow/flight/sql/example/sqlite_tables_schema_batch_reader.h @@ -0,0 +1,58 @@ +// 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. + +#pragma once + +#include + +#include +#include + +#include "arrow/flight/sql/example/sqlite_statement.h" +#include "arrow/flight/sql/example/sqlite_statement_batch_reader.h" +#include "arrow/record_batch.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { + private: + std::shared_ptr reader_; + std::string main_query_; + sqlite3* db_; + + public: + /// Constructor for SqliteTablesWithSchemaBatchReader class + /// \param reader an shared_ptr from a SqliteStatementBatchReader. + /// \param main_query SQL query that originated reader's data. + /// \param db a pointer to the sqlite3 db. + SqliteTablesWithSchemaBatchReader( + std::shared_ptr reader, std::string main_query, + sqlite3* db) + : reader_(std::move(reader)), main_query_(std::move(main_query)), db_(db) {} + + std::shared_ptr schema() const override; + + Status ReadNext(std::shared_ptr* batch) override; +}; + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/server.cc b/cpp/src/arrow/flight/sql/server.cc new file mode 100644 index 00000000000..6d328c07b0e --- /dev/null +++ b/cpp/src/arrow/flight/sql/server.cc @@ -0,0 +1,761 @@ +// 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. + +// Interfaces to use for defining Flight RPC servers. API should be considered +// experimental for now + +#include "arrow/flight/sql/server.h" + +#include + +#include "arrow/buffer.h" +#include "arrow/builder.h" +#include "arrow/flight/sql/FlightSql.pb.h" +#include "arrow/flight/sql/sql_info_internal.h" +#include "arrow/type.h" +#include "arrow/util/checked_cast.h" + +#define PROPERTY_TO_OPTIONAL(COMMAND, PROPERTY) \ + COMMAND.has_##PROPERTY() ? util::make_optional(COMMAND.PROPERTY()) : util::nullopt + +namespace arrow { +namespace flight { +namespace sql { + +namespace pb = arrow::flight::protocol; + +using arrow::internal::checked_cast; +using arrow::internal::checked_pointer_cast; + +namespace { + +arrow::Result ParseCommandGetCrossReference( + const google::protobuf::Any& any) { + pb::sql::CommandGetCrossReference command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetCrossReference."); + } + + GetCrossReference result; + result.pk_table_ref = {PROPERTY_TO_OPTIONAL(command, pk_catalog), + PROPERTY_TO_OPTIONAL(command, pk_db_schema), command.pk_table()}; + result.fk_table_ref = {PROPERTY_TO_OPTIONAL(command, fk_catalog), + PROPERTY_TO_OPTIONAL(command, fk_db_schema), command.fk_table()}; + return result; +} + +arrow::Result ParseCommandGetImportedKeys( + const google::protobuf::Any& any) { + pb::sql::CommandGetImportedKeys command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetImportedKeys."); + } + + GetImportedKeys result; + result.table_ref = {PROPERTY_TO_OPTIONAL(command, catalog), + PROPERTY_TO_OPTIONAL(command, db_schema), command.table()}; + return result; +} + +arrow::Result ParseCommandGetExportedKeys( + const google::protobuf::Any& any) { + pb::sql::CommandGetExportedKeys command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetExportedKeys."); + } + + GetExportedKeys result; + result.table_ref = {PROPERTY_TO_OPTIONAL(command, catalog), + PROPERTY_TO_OPTIONAL(command, db_schema), command.table()}; + return result; +} + +arrow::Result ParseCommandGetPrimaryKeys( + const google::protobuf::Any& any) { + pb::sql::CommandGetPrimaryKeys command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetPrimaryKeys."); + } + + GetPrimaryKeys result; + result.table_ref = {PROPERTY_TO_OPTIONAL(command, catalog), + PROPERTY_TO_OPTIONAL(command, db_schema), command.table()}; + return result; +} + +arrow::Result ParseCommandGetSqlInfo( + const google::protobuf::Any& any, const SqlInfoResultMap& sql_info_id_to_result) { + pb::sql::CommandGetSqlInfo command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetSqlInfo."); + } + + GetSqlInfo result; + if (command.info_size() > 0) { + result.info.reserve(command.info_size()); + result.info.assign(command.info().begin(), command.info().end()); + } else { + result.info.reserve(sql_info_id_to_result.size()); + for (const auto& it : sql_info_id_to_result) { + result.info.push_back(it.first); + } + } + return result; +} + +arrow::Result ParseCommandGetDbSchemas(const google::protobuf::Any& any) { + pb::sql::CommandGetDbSchemas command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetDbSchemas."); + } + + GetDbSchemas result; + result.catalog = PROPERTY_TO_OPTIONAL(command, catalog); + result.db_schema_filter_pattern = + PROPERTY_TO_OPTIONAL(command, db_schema_filter_pattern); + return result; +} + +arrow::Result ParseCommandPreparedStatementQuery( + const google::protobuf::Any& any) { + pb::sql::CommandPreparedStatementQuery command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandPreparedStatementQuery."); + } + + PreparedStatementQuery result; + result.prepared_statement_handle = command.prepared_statement_handle(); + return result; +} + +arrow::Result ParseCommandStatementQuery( + const google::protobuf::Any& any) { + pb::sql::CommandStatementQuery command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandStatementQuery."); + } + + StatementQuery result; + result.query = command.query(); + return result; +} + +arrow::Result ParseCommandGetTables(const google::protobuf::Any& any) { + pb::sql::CommandGetTables command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetTables."); + } + + std::vector table_types(command.table_types_size()); + std::copy(command.table_types().begin(), command.table_types().end(), + table_types.begin()); + + GetTables result; + result.catalog = PROPERTY_TO_OPTIONAL(command, catalog); + result.db_schema_filter_pattern = + PROPERTY_TO_OPTIONAL(command, db_schema_filter_pattern); + result.table_name_filter_pattern = + PROPERTY_TO_OPTIONAL(command, table_name_filter_pattern); + result.table_types = table_types; + result.include_schema = command.include_schema(); + return result; +} + +arrow::Result ParseStatementQueryTicket( + const google::protobuf::Any& any) { + pb::sql::TicketStatementQuery command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack TicketStatementQuery."); + } + + StatementQueryTicket result; + result.statement_handle = command.statement_handle(); + return result; +} + +arrow::Result ParseCommandStatementUpdate( + const google::protobuf::Any& any) { + pb::sql::CommandStatementUpdate command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandStatementUpdate."); + } + + StatementUpdate result; + result.query = command.query(); + return result; +} + +arrow::Result ParseCommandPreparedStatementUpdate( + const google::protobuf::Any& any) { + pb::sql::CommandPreparedStatementUpdate command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandPreparedStatementUpdate."); + } + + PreparedStatementUpdate result; + result.prepared_statement_handle = command.prepared_statement_handle(); + return result; +} + +arrow::Result +ParseActionCreatePreparedStatementRequest(const google::protobuf::Any& any) { + pb::sql::ActionCreatePreparedStatementRequest command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack ActionCreatePreparedStatementRequest."); + } + + ActionCreatePreparedStatementRequest result; + result.query = command.query(); + return result; +} + +arrow::Result +ParseActionClosePreparedStatementRequest(const google::protobuf::Any& any) { + pb::sql::ActionClosePreparedStatementRequest command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack ActionClosePreparedStatementRequest."); + } + + ActionClosePreparedStatementRequest result; + result.prepared_statement_handle = command.prepared_statement_handle(); + return result; +} + +} // namespace + +arrow::Result CreateStatementQueryTicket( + const std::string& statement_handle) { + protocol::sql::TicketStatementQuery ticket_statement_query; + ticket_statement_query.set_statement_handle(statement_handle); + + google::protobuf::Any ticket; + ticket.PackFrom(ticket_statement_query); + + std::string ticket_string; + + if (!ticket.SerializeToString(&ticket_string)) { + return Status::IOError("Invalid ticket."); + } + return ticket_string; +} + +Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, + const FlightDescriptor& request, + std::unique_ptr* info) { + google::protobuf::Any any; + if (!any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size()))) { + return Status::Invalid("Unable to parse command"); + } + + if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(StatementQuery internal_command, + ParseCommandStatementQuery(any)); + ARROW_ASSIGN_OR_RAISE(*info, + GetFlightInfoStatement(context, internal_command, request)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command, + ParseCommandPreparedStatementQuery(any)); + ARROW_ASSIGN_OR_RAISE( + *info, GetFlightInfoPreparedStatement(context, internal_command, request)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(*info, GetFlightInfoCatalogs(context, request)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetDbSchemas internal_command, ParseCommandGetDbSchemas(any)); + ARROW_ASSIGN_OR_RAISE(*info, + GetFlightInfoSchemas(context, internal_command, request)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetTables command, ParseCommandGetTables(any)); + ARROW_ASSIGN_OR_RAISE(*info, GetFlightInfoTables(context, command, request)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(*info, GetFlightInfoTableTypes(context, request)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetSqlInfo internal_command, + ParseCommandGetSqlInfo(any, sql_info_id_to_result_)); + ARROW_ASSIGN_OR_RAISE(*info, + GetFlightInfoSqlInfo(context, internal_command, request)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetPrimaryKeys internal_command, + ParseCommandGetPrimaryKeys(any)); + ARROW_ASSIGN_OR_RAISE(*info, + GetFlightInfoPrimaryKeys(context, internal_command, request)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetExportedKeys internal_command, + ParseCommandGetExportedKeys(any)); + ARROW_ASSIGN_OR_RAISE(*info, + GetFlightInfoExportedKeys(context, internal_command, request)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetImportedKeys internal_command, + ParseCommandGetImportedKeys(any)); + ARROW_ASSIGN_OR_RAISE(*info, + GetFlightInfoImportedKeys(context, internal_command, request)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetCrossReference internal_command, + ParseCommandGetCrossReference(any)); + ARROW_ASSIGN_OR_RAISE( + *info, GetFlightInfoCrossReference(context, internal_command, request)); + return Status::OK(); + } + + return Status::Invalid("The defined request is invalid."); +} + +Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket& request, + std::unique_ptr* stream) { + google::protobuf::Any any; + + if (!any.ParseFromArray(request.ticket.data(), + static_cast(request.ticket.size()))) { + return Status::Invalid("Unable to parse ticket."); + } + + if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(StatementQueryTicket command, ParseStatementQueryTicket(any)); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetStatement(context, command)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command, + ParseCommandPreparedStatementQuery(any)); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetPreparedStatement(context, internal_command)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(*stream, DoGetCatalogs(context)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetDbSchemas internal_command, ParseCommandGetDbSchemas(any)); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetDbSchemas(context, internal_command)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetTables command, ParseCommandGetTables(any)); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetTables(context, command)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(*stream, DoGetTableTypes(context)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetSqlInfo internal_command, + ParseCommandGetSqlInfo(any, sql_info_id_to_result_)); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetSqlInfo(context, internal_command)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetPrimaryKeys internal_command, + ParseCommandGetPrimaryKeys(any)); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetPrimaryKeys(context, internal_command)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetExportedKeys internal_command, + ParseCommandGetExportedKeys(any)); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetExportedKeys(context, internal_command)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetImportedKeys internal_command, + ParseCommandGetImportedKeys(any)); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetImportedKeys(context, internal_command)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetCrossReference internal_command, + ParseCommandGetCrossReference(any)); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetCrossReference(context, internal_command)); + return Status::OK(); + } + + return Status::Invalid("The defined request is invalid."); +} + +Status FlightSqlServerBase::DoPut(const ServerCallContext& context, + std::unique_ptr reader, + std::unique_ptr writer) { + const FlightDescriptor& request = reader->descriptor(); + + google::protobuf::Any any; + if (!any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size()))) { + return Status::Invalid("Unable to parse command."); + } + + if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(StatementUpdate internal_command, + ParseCommandStatementUpdate(any)); + ARROW_ASSIGN_OR_RAISE(auto record_count, + DoPutCommandStatementUpdate(context, internal_command)) + + pb::sql::DoPutUpdateResult result; + result.set_record_count(record_count); + + const auto buffer = Buffer::FromString(result.SerializeAsString()); + ARROW_RETURN_NOT_OK(writer->WriteMetadata(*buffer)); + + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command, + ParseCommandPreparedStatementQuery(any)); + return DoPutPreparedStatementQuery(context, internal_command, reader.get(), + writer.get()); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(PreparedStatementUpdate internal_command, + ParseCommandPreparedStatementUpdate(any)); + ARROW_ASSIGN_OR_RAISE(auto record_count, DoPutPreparedStatementUpdate( + context, internal_command, reader.get())) + + pb::sql::DoPutUpdateResult result; + result.set_record_count(record_count); + + const auto buffer = Buffer::FromString(result.SerializeAsString()); + ARROW_RETURN_NOT_OK(writer->WriteMetadata(*buffer)); + + return Status::OK(); + } + + return Status::Invalid("The defined request is invalid."); +} + +Status FlightSqlServerBase::ListActions(const ServerCallContext& context, + std::vector* actions) { + *actions = {FlightSqlServerBase::kCreatePreparedStatementActionType, + FlightSqlServerBase::kClosePreparedStatementActionType}; + return Status::OK(); +} + +Status FlightSqlServerBase::DoAction(const ServerCallContext& context, + const Action& action, + std::unique_ptr* result_stream) { + if (action.type == FlightSqlServerBase::kCreatePreparedStatementActionType.type) { + google::protobuf::Any any_command; + if (!any_command.ParseFromArray(action.body->data(), + static_cast(action.body->size()))) { + return Status::Invalid("Unable to parse action."); + } + + ARROW_ASSIGN_OR_RAISE(ActionCreatePreparedStatementRequest internal_command, + ParseActionCreatePreparedStatementRequest(any_command)); + ARROW_ASSIGN_OR_RAISE(auto result, CreatePreparedStatement(context, internal_command)) + + pb::sql::ActionCreatePreparedStatementResult action_result; + action_result.set_prepared_statement_handle(result.prepared_statement_handle); + if (result.dataset_schema != nullptr) { + ARROW_ASSIGN_OR_RAISE(auto serialized_dataset_schema, + ipc::SerializeSchema(*result.dataset_schema)) + action_result.set_dataset_schema(serialized_dataset_schema->ToString()); + } + if (result.parameter_schema != nullptr) { + ARROW_ASSIGN_OR_RAISE(auto serialized_parameter_schema, + ipc::SerializeSchema(*result.parameter_schema)) + action_result.set_parameter_schema(serialized_parameter_schema->ToString()); + } + + google::protobuf::Any any; + any.PackFrom(action_result); + + auto buf = Buffer::FromString(any.SerializeAsString()); + *result_stream = std::unique_ptr(new SimpleResultStream({Result{buf}})); + + return Status::OK(); + } else if (action.type == FlightSqlServerBase::kClosePreparedStatementActionType.type) { + google::protobuf::Any any; + if (!any.ParseFromArray(action.body->data(), static_cast(action.body->size()))) { + return Status::Invalid("Unable to parse action."); + } + + ARROW_ASSIGN_OR_RAISE(ActionClosePreparedStatementRequest internal_command, + ParseActionClosePreparedStatementRequest(any)); + + ARROW_RETURN_NOT_OK(ClosePreparedStatement(context, internal_command)); + + // Need to instantiate a ResultStream, otherwise clients can not wait for completion. + *result_stream = std::unique_ptr(new SimpleResultStream({})); + return Status::OK(); + } + return Status::Invalid("The defined request is invalid."); +} + +arrow::Result> FlightSqlServerBase::GetFlightInfoCatalogs( + const ServerCallContext& context, const FlightDescriptor& descriptor) { + return Status::NotImplemented("GetFlightInfoCatalogs not implemented"); +} + +arrow::Result> FlightSqlServerBase::DoGetCatalogs( + const ServerCallContext& context) { + return Status::NotImplemented("DoGetCatalogs not implemented"); +} + +arrow::Result> FlightSqlServerBase::GetFlightInfoStatement( + const ServerCallContext& context, const StatementQuery& command, + const FlightDescriptor& descriptor) { + return Status::NotImplemented("GetFlightInfoStatement not implemented"); +} + +arrow::Result> FlightSqlServerBase::DoGetStatement( + const ServerCallContext& context, const StatementQueryTicket& command) { + return Status::NotImplemented("DoGetStatement not implemented"); +} + +arrow::Result> +FlightSqlServerBase::GetFlightInfoPreparedStatement(const ServerCallContext& context, + const PreparedStatementQuery& command, + const FlightDescriptor& descriptor) { + return Status::NotImplemented("GetFlightInfoPreparedStatement not implemented"); +} + +arrow::Result> +FlightSqlServerBase::DoGetPreparedStatement(const ServerCallContext& context, + const PreparedStatementQuery& command) { + return Status::NotImplemented("DoGetPreparedStatement not implemented"); +} + +arrow::Result> FlightSqlServerBase::GetFlightInfoSqlInfo( + const ServerCallContext& context, const GetSqlInfo& command, + const FlightDescriptor& descriptor) { + if (sql_info_id_to_result_.empty()) { + return Status::KeyError("No SQL information available."); + } + + std::vector endpoints{FlightEndpoint{{descriptor.cmd}, {}}}; + ARROW_ASSIGN_OR_RAISE(auto result, FlightInfo::Make(*SqlSchema::GetSqlInfoSchema(), + descriptor, endpoints, -1, -1)) + + return std::unique_ptr(new FlightInfo(result)); +} + +void FlightSqlServerBase::RegisterSqlInfo(int32_t id, const SqlInfoResult& result) { + sql_info_id_to_result_[id] = result; +} + +arrow::Result> FlightSqlServerBase::DoGetSqlInfo( + const ServerCallContext& context, const GetSqlInfo& command) { + MemoryPool* memory_pool = default_memory_pool(); + UInt32Builder name_field_builder(memory_pool); + std::unique_ptr value_field_builder; + const auto& value_field_type = checked_pointer_cast( + SqlSchema::GetSqlInfoSchema()->fields()[1]->type()); + ARROW_RETURN_NOT_OK(MakeBuilder(memory_pool, value_field_type, &value_field_builder)); + + internal::SqlInfoResultAppender sql_info_result_appender( + checked_cast(value_field_builder.get())); + + // Populate both name_field_builder and value_field_builder for each element + // on command.info. + // value_field_builder is populated differently depending on the data type (as it is + // a DenseUnionBuilder). The population for each data type is implemented on + // internal::SqlInfoResultAppender. + for (const auto& info : command.info) { + const auto it = sql_info_id_to_result_.find(info); + if (it == sql_info_id_to_result_.end()) { + return Status::KeyError("No information for SQL info number ", info); + } + ARROW_RETURN_NOT_OK(name_field_builder.Append(info)); + ARROW_RETURN_NOT_OK(arrow::util::visit(sql_info_result_appender, it->second)); + } + + std::shared_ptr name; + ARROW_RETURN_NOT_OK(name_field_builder.Finish(&name)); + std::shared_ptr value; + ARROW_RETURN_NOT_OK(value_field_builder->Finish(&value)); + + auto row_count = static_cast(command.info.size()); + const std::shared_ptr& batch = + RecordBatch::Make(SqlSchema::GetSqlInfoSchema(), row_count, {name, value}); + ARROW_ASSIGN_OR_RAISE(const auto reader, RecordBatchReader::Make({batch})); + + return std::unique_ptr(new RecordBatchStream(reader)); +} + +arrow::Result> FlightSqlServerBase::GetFlightInfoSchemas( + const ServerCallContext& context, const GetDbSchemas& command, + const FlightDescriptor& descriptor) { + return Status::NotImplemented("GetFlightInfoSchemas not implemented"); +} + +arrow::Result> FlightSqlServerBase::DoGetDbSchemas( + const ServerCallContext& context, const GetDbSchemas& command) { + return Status::NotImplemented("DoGetDbSchemas not implemented"); +} + +arrow::Result> FlightSqlServerBase::GetFlightInfoTables( + const ServerCallContext& context, const GetTables& command, + const FlightDescriptor& descriptor) { + return Status::NotImplemented("GetFlightInfoTables not implemented"); +} + +arrow::Result> FlightSqlServerBase::DoGetTables( + const ServerCallContext& context, const GetTables& command) { + return Status::NotImplemented("DoGetTables not implemented"); +} + +arrow::Result> FlightSqlServerBase::GetFlightInfoTableTypes( + const ServerCallContext& context, const FlightDescriptor& descriptor) { + return Status::NotImplemented("GetFlightInfoTableTypes not implemented"); +} + +arrow::Result> FlightSqlServerBase::DoGetTableTypes( + const ServerCallContext& context) { + return Status::NotImplemented("DoGetTableTypes not implemented"); +} + +arrow::Result> FlightSqlServerBase::GetFlightInfoPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command, + const FlightDescriptor& descriptor) { + return Status::NotImplemented("GetFlightInfoPrimaryKeys not implemented"); +} + +arrow::Result> FlightSqlServerBase::DoGetPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command) { + return Status::NotImplemented("DoGetPrimaryKeys not implemented"); +} + +arrow::Result> FlightSqlServerBase::GetFlightInfoExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command, + const FlightDescriptor& descriptor) { + return Status::NotImplemented("GetFlightInfoExportedKeys not implemented"); +} + +arrow::Result> FlightSqlServerBase::DoGetExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command) { + return Status::NotImplemented("DoGetExportedKeys not implemented"); +} + +arrow::Result> FlightSqlServerBase::GetFlightInfoImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command, + const FlightDescriptor& descriptor) { + return Status::NotImplemented("GetFlightInfoImportedKeys not implemented"); +} + +arrow::Result> FlightSqlServerBase::DoGetImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command) { + return Status::NotImplemented("DoGetImportedKeys not implemented"); +} + +arrow::Result> +FlightSqlServerBase::GetFlightInfoCrossReference(const ServerCallContext& context, + const GetCrossReference& command, + const FlightDescriptor& descriptor) { + return Status::NotImplemented("GetFlightInfoCrossReference not implemented"); +} + +arrow::Result> FlightSqlServerBase::DoGetCrossReference( + const ServerCallContext& context, const GetCrossReference& command) { + return Status::NotImplemented("DoGetCrossReference not implemented"); +} + +arrow::Result +FlightSqlServerBase::CreatePreparedStatement( + const ServerCallContext& context, + const ActionCreatePreparedStatementRequest& request) { + return Status::NotImplemented("CreatePreparedStatement not implemented"); +} + +Status FlightSqlServerBase::ClosePreparedStatement( + const ServerCallContext& context, + const ActionClosePreparedStatementRequest& request) { + return Status::NotImplemented("ClosePreparedStatement not implemented"); +} + +Status FlightSqlServerBase::DoPutPreparedStatementQuery( + const ServerCallContext& context, const PreparedStatementQuery& command, + FlightMessageReader* reader, FlightMetadataWriter* writer) { + return Status::NotImplemented("DoPutPreparedStatementQuery not implemented"); +} + +arrow::Result FlightSqlServerBase::DoPutPreparedStatementUpdate( + const ServerCallContext& context, const PreparedStatementUpdate& command, + FlightMessageReader* reader) { + return Status::NotImplemented("DoPutPreparedStatementUpdate not implemented"); +} + +arrow::Result FlightSqlServerBase::DoPutCommandStatementUpdate( + const ServerCallContext& context, const StatementUpdate& command) { + return Status::NotImplemented("DoPutCommandStatementUpdate not implemented"); +} + +std::shared_ptr SqlSchema::GetCatalogsSchema() { + return arrow::schema({field("catalog_name", utf8())}); +} + +std::shared_ptr SqlSchema::GetDbSchemasSchema() { + return arrow::schema( + {field("catalog_name", utf8()), field("db_schema_name", utf8(), false)}); +} + +std::shared_ptr SqlSchema::GetTablesSchema() { + return arrow::schema({field("catalog_name", utf8()), field("db_schema_name", utf8()), + field("table_name", utf8()), field("table_type", utf8())}); +} + +std::shared_ptr SqlSchema::GetTablesSchemaWithIncludedSchema() { + return arrow::schema({field("catalog_name", utf8()), field("db_schema_name", utf8()), + field("table_name", utf8()), field("table_type", utf8()), + field("table_schema", binary())}); +} + +std::shared_ptr SqlSchema::GetTableTypesSchema() { + return arrow::schema({field("table_type", utf8())}); +} + +std::shared_ptr SqlSchema::GetPrimaryKeysSchema() { + return arrow::schema({field("catalog_name", utf8()), field("db_schema_name", utf8()), + field("table_name", utf8()), field("column_name", utf8()), + field("key_sequence", int64()), field("key_name", utf8())}); +} + +std::shared_ptr GetImportedExportedKeysAndCrossReferenceSchema() { + return arrow::schema( + {field("pk_catalog_name", utf8(), true), field("pk_db_schema_name", utf8(), true), + field("pk_table_name", utf8(), false), field("pk_column_name", utf8(), false), + field("fk_catalog_name", utf8(), true), field("fk_db_schema_name", utf8(), true), + field("fk_table_name", utf8(), false), field("fk_column_name", utf8(), false), + field("key_sequence", int32(), false), field("fk_key_name", utf8(), true), + field("pk_key_name", utf8(), true), field("update_rule", uint8(), false), + field("delete_rule", uint8(), false)}); +} + +std::shared_ptr SqlSchema::GetImportedKeysSchema() { + return GetImportedExportedKeysAndCrossReferenceSchema(); +} + +std::shared_ptr SqlSchema::GetExportedKeysSchema() { + return GetImportedExportedKeysAndCrossReferenceSchema(); +} + +std::shared_ptr SqlSchema::GetCrossReferenceSchema() { + return GetImportedExportedKeysAndCrossReferenceSchema(); +} + +std::shared_ptr SqlSchema::GetSqlInfoSchema() { + return arrow::schema({field("name", uint32(), false), + field("value", + dense_union({field("string_value", utf8(), false), + field("bool_value", boolean(), false), + field("bigint_value", int64(), false), + field("int32_bitmask", int32(), false), + field("string_list", list(utf8()), false), + field("int32_to_int32_list_map", + map(int32(), list(int32())), false)}), + false)}); +} + +} // namespace sql +} // namespace flight +} // namespace arrow + +#undef PROPERTY_TO_OPTIONAL diff --git a/cpp/src/arrow/flight/sql/server.h b/cpp/src/arrow/flight/sql/server.h new file mode 100644 index 00000000000..1d6101683c1 --- /dev/null +++ b/cpp/src/arrow/flight/sql/server.h @@ -0,0 +1,443 @@ +// 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. + +// Interfaces to use for defining Flight RPC servers. API should be considered +// experimental for now + +#pragma once + +#include +#include +#include + +#include "arrow/flight/server.h" +#include "arrow/flight/sql/server.h" +#include "arrow/flight/sql/types.h" +#include "arrow/util/optional.h" + +namespace arrow { +namespace flight { +namespace sql { + +struct StatementQuery { + std::string query; +}; + +struct StatementUpdate { + std::string query; +}; + +struct StatementQueryTicket { + std::string statement_handle; +}; + +struct PreparedStatementQuery { + std::string prepared_statement_handle; +}; + +struct PreparedStatementUpdate { + std::string prepared_statement_handle; +}; + +struct GetSqlInfo { + std::vector info; +}; + +struct GetDbSchemas { + util::optional catalog; + util::optional db_schema_filter_pattern; +}; + +struct GetTables { + util::optional catalog; + util::optional db_schema_filter_pattern; + util::optional table_name_filter_pattern; + std::vector table_types; + bool include_schema; +}; + +struct GetPrimaryKeys { + TableRef table_ref; +}; + +struct GetExportedKeys { + TableRef table_ref; +}; + +struct GetImportedKeys { + TableRef table_ref; +}; + +struct GetCrossReference { + TableRef pk_table_ref; + TableRef fk_table_ref; +}; + +struct ActionCreatePreparedStatementRequest { + std::string query; +}; + +struct ActionClosePreparedStatementRequest { + std::string prepared_statement_handle; +}; + +struct ActionCreatePreparedStatementResult { + std::shared_ptr dataset_schema; + std::shared_ptr parameter_schema; + std::string prepared_statement_handle; +}; + +/// \brief A utility function to create a ticket (a opaque binary token that the server +/// uses to identify this query) for a statement query. +/// Intended for Flight SQL server implementations. +/// \param[in] statement_handle The statement handle that will originate the ticket. +/// \return The parsed ticket as an string. +arrow::Result CreateStatementQueryTicket( + const std::string& statement_handle); + +class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { + private: + SqlInfoResultMap sql_info_id_to_result_; + + public: + Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, + std::unique_ptr* info) override; + + Status DoGet(const ServerCallContext& context, const Ticket& request, + std::unique_ptr* stream) override; + + Status DoPut(const ServerCallContext& context, + std::unique_ptr reader, + std::unique_ptr writer) override; + + const ActionType kCreatePreparedStatementActionType = + ActionType{"CreatePreparedStatement", + "Creates a reusable prepared statement resource on the server.\n" + "Request Message: ActionCreatePreparedStatementRequest\n" + "Response Message: ActionCreatePreparedStatementResult"}; + const ActionType kClosePreparedStatementActionType = + ActionType{"ClosePreparedStatement", + "Closes a reusable prepared statement resource on the server.\n" + "Request Message: ActionClosePreparedStatementRequest\n" + "Response Message: N/A"}; + + Status ListActions(const ServerCallContext& context, + std::vector* actions) override; + + Status DoAction(const ServerCallContext& context, const Action& action, + std::unique_ptr* result) override; + + /// \brief Get a FlightInfo for executing a SQL query. + /// \param[in] context Per-call context. + /// \param[in] command The StatementQuery object containing the SQL statement. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \return The FlightInfo describing where to access the dataset. + virtual arrow::Result> GetFlightInfoStatement( + const ServerCallContext& context, const StatementQuery& command, + const FlightDescriptor& descriptor); + + /// \brief Get a FlightDataStream containing the query results. + /// \param[in] context Per-call context. + /// \param[in] command The StatementQueryTicket containing the statement handle. + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetStatement( + const ServerCallContext& context, const StatementQueryTicket& command); + + /// \brief Get a FlightInfo for executing an already created prepared statement. + /// \param[in] context Per-call context. + /// \param[in] command The PreparedStatementQuery object containing the + /// prepared statement handle. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \return The FlightInfo describing where to access the + /// dataset. + virtual arrow::Result> GetFlightInfoPreparedStatement( + const ServerCallContext& context, const PreparedStatementQuery& command, + const FlightDescriptor& descriptor); + + /// \brief Get a FlightDataStream containing the prepared statement query results. + /// \param[in] context Per-call context. + /// \param[in] command The PreparedStatementQuery object containing the + /// prepared statement handle. + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetPreparedStatement( + const ServerCallContext& context, const PreparedStatementQuery& command); + + /// \brief Get a FlightInfo for listing catalogs. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \return The FlightInfo describing where to access the dataset. + virtual arrow::Result> GetFlightInfoCatalogs( + const ServerCallContext& context, const FlightDescriptor& descriptor); + + /// \brief Get a FlightDataStream containing the list of catalogs. + /// \param[in] context Per-call context. + /// \return An interface for sending data back to the client. + virtual arrow::Result> DoGetCatalogs( + const ServerCallContext& context); + + /// \brief Get a FlightInfo for retrieving other information (See SqlInfo). + /// \param[in] context Per-call context. + /// \param[in] command The GetSqlInfo object containing the list of SqlInfo + /// to be returned. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \return The FlightInfo describing where to access the dataset. + virtual arrow::Result> GetFlightInfoSqlInfo( + const ServerCallContext& context, const GetSqlInfo& command, + const FlightDescriptor& descriptor); + + /// \brief Get a FlightDataStream containing the list of SqlInfo results. + /// \param[in] context Per-call context. + /// \param[in] command The GetSqlInfo object containing the list of SqlInfo + /// to be returned. + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetSqlInfo( + const ServerCallContext& context, const GetSqlInfo& command); + + /// \brief Get a FlightInfo for listing schemas. + /// \param[in] context Per-call context. + /// \param[in] command The GetDbSchemas object which may contain filters for + /// catalog and schema name. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \return The FlightInfo describing where to access the dataset. + virtual arrow::Result> GetFlightInfoSchemas( + const ServerCallContext& context, const GetDbSchemas& command, + const FlightDescriptor& descriptor); + + /// \brief Get a FlightDataStream containing the list of schemas. + /// \param[in] context Per-call context. + /// \param[in] command The GetDbSchemas object which may contain filters for + /// catalog and schema name. + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetDbSchemas( + const ServerCallContext& context, const GetDbSchemas& command); + + ///\brief Get a FlightInfo for listing tables. + /// \param[in] context Per-call context. + /// \param[in] command The GetTables object which may contain filters for + /// catalog, schema and table names. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \return The FlightInfo describing where to access the dataset. + virtual arrow::Result> GetFlightInfoTables( + const ServerCallContext& context, const GetTables& command, + const FlightDescriptor& descriptor); + + /// \brief Get a FlightDataStream containing the list of tables. + /// \param[in] context Per-call context. + /// \param[in] command The GetTables object which may contain filters for + /// catalog, schema and table names. + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetTables( + const ServerCallContext& context, const GetTables& command); + + /// \brief Get a FlightInfo to extract information about the table types. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \return The FlightInfo describing where to access the + /// dataset. + virtual arrow::Result> GetFlightInfoTableTypes( + const ServerCallContext& context, const FlightDescriptor& descriptor); + + /// \brief Get a FlightDataStream containing the data related to the table types. + /// \param[in] context Per-call context. + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetTableTypes( + const ServerCallContext& context); + + /// \brief Get a FlightInfo to extract information about primary and foreign keys. + /// \param[in] context Per-call context. + /// \param[in] command The GetPrimaryKeys object with necessary information + /// to execute the request. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \return The FlightInfo describing where to access the + /// dataset. + virtual arrow::Result> GetFlightInfoPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command, + const FlightDescriptor& descriptor); + + /// \brief Get a FlightDataStream containing the data related to the primary and + /// foreign + /// keys. + /// \param[in] context Per-call context. + /// \param[in] command The GetPrimaryKeys object with necessary information + /// to execute the request. + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command); + + /// \brief Get a FlightInfo to extract information about foreign and primary keys. + /// \param[in] context Per-call context. + /// \param[in] command The GetExportedKeys object with necessary information + /// to execute the request. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \return The FlightInfo describing where to access the + /// dataset. + virtual arrow::Result> GetFlightInfoExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command, + const FlightDescriptor& descriptor); + + /// \brief Get a FlightDataStream containing the data related to the foreign and + /// primary + /// keys. + /// \param[in] context Per-call context. + /// \param[in] command The GetExportedKeys object with necessary information + /// to execute the request. + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command); + + /// \brief Get a FlightInfo to extract information about foreign and primary keys. + /// \param[in] context Per-call context. + /// \param[in] command The GetImportedKeys object with necessary information + /// to execute the request. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \return The FlightInfo describing where to access the + /// dataset. + virtual arrow::Result> GetFlightInfoImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command, + const FlightDescriptor& descriptor); + + /// \brief Get a FlightDataStream containing the data related to the foreign and + /// primary keys. + /// \param[in] context Per-call context. + /// \param[in] command The GetImportedKeys object with necessary information + /// to execute the request. + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command); + + /// \brief Get a FlightInfo to extract information about foreign and primary keys. + /// \param[in] context Per-call context. + /// \param[in] command The GetCrossReference object with necessary + /// information + /// to execute the request. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \return The FlightInfo describing where to access the + /// dataset. + virtual arrow::Result> GetFlightInfoCrossReference( + const ServerCallContext& context, const GetCrossReference& command, + const FlightDescriptor& descriptor); + + /// \brief Get a FlightDataStream containing the data related to the foreign and + /// primary keys. + /// \param[in] context Per-call context. + /// \param[in] command The GetCrossReference object with necessary information + /// to execute the request. + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetCrossReference( + const ServerCallContext& context, const GetCrossReference& command); + + /// \brief Execute an update SQL statement. + /// \param[in] context The call context. + /// \param[in] command The StatementUpdate object containing the SQL statement. + /// \return The changed record count. + virtual arrow::Result DoPutCommandStatementUpdate( + const ServerCallContext& context, const StatementUpdate& command); + + /// \brief Create a prepared statement from given SQL statement. + /// \param[in] context The call context. + /// \param[in] request The ActionCreatePreparedStatementRequest object containing the + /// SQL statement. + /// \return A ActionCreatePreparedStatementResult containing the dataset + /// and parameter schemas and a handle for created statement. + virtual arrow::Result CreatePreparedStatement( + const ServerCallContext& context, + const ActionCreatePreparedStatementRequest& request); + + /// \brief Close a prepared statement. + /// \param[in] context The call context. + /// \param[in] request The ActionClosePreparedStatementRequest object containing the + /// prepared statement handle. + virtual Status ClosePreparedStatement( + const ServerCallContext& context, + const ActionClosePreparedStatementRequest& request); + + /// \brief Bind parameters to given prepared statement. + /// \param[in] context The call context. + /// \param[in] command The PreparedStatementQuery object containing the + /// prepared statement handle. + /// \param[in] reader A sequence of uploaded record batches. + /// \param[in] writer Send metadata back to the client. + virtual Status DoPutPreparedStatementQuery(const ServerCallContext& context, + const PreparedStatementQuery& command, + FlightMessageReader* reader, + FlightMetadataWriter* writer); + + /// \brief Execute an update SQL prepared statement. + /// \param[in] context The call context. + /// \param[in] command The PreparedStatementUpdate object containing the + /// prepared statement handle. + /// \param[in] reader a sequence of uploaded record batches. + /// \return The changed record count. + virtual arrow::Result DoPutPreparedStatementUpdate( + const ServerCallContext& context, const PreparedStatementUpdate& command, + FlightMessageReader* reader); + + /// \brief Register a new SqlInfo result, making it available when calling GetSqlInfo. + /// \param[in] id the SqlInfo identifier. + /// \param[in] result the result. + void RegisterSqlInfo(int32_t id, const SqlInfoResult& result); +}; + +/// \brief Auxiliary class containing all Schemas used on Flight SQL. +class ARROW_EXPORT SqlSchema { + public: + /// \brief Get the Schema used on GetCatalogs response. + /// \return The default schema template. + static std::shared_ptr GetCatalogsSchema(); + + /// \brief Get the Schema used on GetDbSchemas response. + /// \return The default schema template. + static std::shared_ptr GetDbSchemasSchema(); + + /// \brief Get the Schema used on GetTables response when included schema + /// flags is set to false. + /// \return The default schema template. + static std::shared_ptr GetTablesSchema(); + + /// \brief Get the Schema used on GetTables response when included schema + /// flags is set to true. + /// \return The default schema template. + static std::shared_ptr GetTablesSchemaWithIncludedSchema(); + + /// \brief Get the Schema used on GetTableTypes response. + /// \return The default schema template. + static std::shared_ptr GetTableTypesSchema(); + + /// \brief Get the Schema used on GetPrimaryKeys response when included schema + /// flags is set to true. + /// \return The default schema template. + static std::shared_ptr GetPrimaryKeysSchema(); + + /// \brief Get the Schema used on GetImportedKeys response. + /// \return The default schema template. + static std::shared_ptr GetExportedKeysSchema(); + + /// \brief Get the Schema used on GetImportedKeys response. + /// \return The default schema template. + static std::shared_ptr GetImportedKeysSchema(); + + /// \brief Get the Schema used on GetCrossReference response. + /// \return The default schema template. + static std::shared_ptr GetCrossReferenceSchema(); + + /// \brief Get the Schema used on GetSqlInfo response. + /// \return The default schema template. + static std::shared_ptr GetSqlInfoSchema(); +}; +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/server_test.cc b/cpp/src/arrow/flight/sql/server_test.cc new file mode 100644 index 00000000000..8dfea7a013e --- /dev/null +++ b/cpp/src/arrow/flight/sql/server_test.cc @@ -0,0 +1,767 @@ +// 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. + +#include "arrow/flight/sql/server.h" + +#include +#include +#include + +#include +#include + +#include "arrow/flight/api.h" +#include "arrow/flight/sql/api.h" +#include "arrow/flight/sql/example/sqlite_server.h" +#include "arrow/flight/sql/example/sqlite_sql_info.h" +#include "arrow/flight/test_util.h" +#include "arrow/flight/types.h" +#include "arrow/testing/gtest_util.h" + +using ::testing::_; +using ::testing::Ref; + +using arrow::internal::checked_cast; + +namespace arrow { +namespace flight { +namespace sql { + +/// \brief Auxiliary variant visitor used to assert that GetSqlInfo's values are +/// correctly placed on its DenseUnionArray +class SqlInfoDenseUnionValidator { + private: + const DenseUnionScalar& data; + + public: + /// \brief Asserts that the current DenseUnionScalar equals to given string value + void operator()(const std::string& string_value) const { + const auto& scalar = checked_cast(*data.value); + ASSERT_EQ(string_value, scalar.ToString()); + } + + /// \brief Asserts that the current DenseUnionScalar equals to given bool value + void operator()(const bool bool_value) const { + const auto& scalar = checked_cast(*data.value); + ASSERT_EQ(bool_value, scalar.value); + } + + /// \brief Asserts that the current DenseUnionScalar equals to given int64_t value + void operator()(const int64_t bigint_value) const { + const auto& scalar = checked_cast(*data.value); + ASSERT_EQ(bigint_value, scalar.value); + } + + /// \brief Asserts that the current DenseUnionScalar equals to given int32_t value + void operator()(const int32_t int32_bitmask) const { + const auto& scalar = checked_cast(*data.value); + ASSERT_EQ(int32_bitmask, scalar.value); + } + + /// \brief Asserts that the current DenseUnionScalar equals to given string list + void operator()(const std::vector& string_list) const { + const auto& array = checked_cast( + *(checked_cast(*data.value).value)); + + ASSERT_EQ(string_list.size(), array.length()); + + for (size_t index = 0; index < string_list.size(); index++) { + ASSERT_EQ(string_list[index], array.GetString(index)); + } + } + + /// \brief Asserts that the current DenseUnionScalar equals to given int32 to int32 list + /// map. + void operator()(const std::unordered_map>& + int32_to_int32_list) const { + const auto& struct_array = checked_cast( + *checked_cast(*data.value).value); + const auto& keys = checked_cast(*struct_array.field(0)); + const auto& values = checked_cast(*struct_array.field(1)); + + // Assert that the given map has the right size + ASSERT_EQ(int32_to_int32_list.size(), keys.length()); + + // For each element on given MapScalar, assert it matches the argument + for (int i = 0; i < keys.length(); i++) { + ASSERT_OK_AND_ASSIGN(const auto& key_scalar, keys.GetScalar(i)); + int32_t sql_info_id = checked_cast(*key_scalar).value; + + // Assert the key (SqlInfo id) exists + ASSERT_TRUE(int32_to_int32_list.count(sql_info_id)); + + const std::vector& expected_int32_list = + int32_to_int32_list.at(sql_info_id); + + // Assert the value (int32 list) has the correct size + ASSERT_EQ(expected_int32_list.size(), values.value_length(i)); + + // For each element on current ListScalar, assert it matches with the argument + for (size_t j = 0; j < expected_int32_list.size(); j++) { + ASSERT_OK_AND_ASSIGN(auto list_item_scalar, + values.values()->GetScalar(values.value_offset(i) + j)); + const auto& list_item = checked_cast(*list_item_scalar).value; + ASSERT_EQ(expected_int32_list[j], list_item); + } + } + } + + explicit SqlInfoDenseUnionValidator(const DenseUnionScalar& data) : data(data) {} + + SqlInfoDenseUnionValidator(const SqlInfoDenseUnionValidator&) = delete; + SqlInfoDenseUnionValidator(SqlInfoDenseUnionValidator&&) = delete; + SqlInfoDenseUnionValidator& operator=(const SqlInfoDenseUnionValidator&) = delete; +}; + +class TestFlightSqlServer : public ::testing::Test { + public: + std::unique_ptr sql_client; + + arrow::Result ExecuteCountQuery(const std::string& query) { + ARROW_ASSIGN_OR_RAISE(auto flight_info, sql_client->Execute({}, query)); + + ARROW_ASSIGN_OR_RAISE(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr table; + ARROW_RETURN_NOT_OK(stream->ReadAll(&table)); + + const std::shared_ptr& result_array = table->column(0)->chunk(0); + ARROW_ASSIGN_OR_RAISE(auto count_scalar, result_array->GetScalar(0)); + + return reinterpret_cast(*count_scalar).value; + } + + protected: + void SetUp() override { + port = GetListenPort(); + server_thread.reset(new std::thread([&]() { RunServer(); })); + + std::unique_lock lk(server_ready_m); + server_ready_cv.wait(lk); + + std::stringstream ss; + ss << "grpc://localhost:" << port; + std::string uri = ss.str(); + + std::unique_ptr client; + Location location; + ASSERT_OK(Location::Parse(uri, &location)); + ASSERT_OK(FlightClient::Connect(location, &client)); + + sql_client.reset(new FlightSqlClient(std::move(client))); + } + + void TearDown() override { + sql_client.reset(); + + ASSERT_OK(server->Shutdown()); + server_thread->join(); + server_thread.reset(); + } + + private: + int port; + std::shared_ptr server; + std::unique_ptr server_thread; + std::condition_variable server_ready_cv; + std::mutex server_ready_m; + + void RunServer() { + arrow::flight::Location location; + ARROW_CHECK_OK(arrow::flight::Location::ForGrpcTcp("localhost", port, &location)); + arrow::flight::FlightServerOptions options(location); + + ARROW_CHECK_OK(example::SQLiteFlightSqlServer::Create().Value(&server)); + + ARROW_CHECK_OK(server->Init(options)); + // Exit with a clean error code (0) on SIGTERM + ARROW_CHECK_OK(server->SetShutdownOnSignals({SIGTERM})); + + server_ready_cv.notify_all(); + ARROW_CHECK_OK(server->Serve()); + } +}; + +TEST_F(TestFlightSqlServer, TestCommandStatementQuery) { + ASSERT_OK_AND_ASSIGN(auto flight_info, + sql_client->Execute({}, "SELECT * FROM intTable")); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const std::shared_ptr& expected_schema = + arrow::schema({arrow::field("id", int64()), arrow::field("keyName", utf8()), + arrow::field("value", int64()), arrow::field("foreignId", int64())}); + + const auto id_array = ArrayFromJSON(int64(), R"([1, 2, 3, 4])"); + const auto keyname_array = + ArrayFromJSON(utf8(), R"(["one", "zero", "negative one", null])"); + const auto value_array = ArrayFromJSON(int64(), R"([1, 0, -1, null])"); + const auto foreignId_array = ArrayFromJSON(int64(), R"([1, 1, 1, null])"); + + const std::shared_ptr
& expected_table = Table::Make( + expected_schema, {id_array, keyname_array, value_array, foreignId_array}); + + AssertTablesEqual(*expected_table, *table); +} + +TEST_F(TestFlightSqlServer, TestCommandGetTables) { + FlightCallOptions options = {}; + std::string* catalog = nullptr; + std::string* schema_filter_pattern = nullptr; + std::string* table_filter_pattern = nullptr; + bool include_schema = false; + std::vector* table_types = nullptr; + + ASSERT_OK_AND_ASSIGN( + auto flight_info, + sql_client->GetTables(options, catalog, schema_filter_pattern, table_filter_pattern, + include_schema, table_types)); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + ASSERT_OK_AND_ASSIGN(auto catalog_name, MakeArrayOfNull(utf8(), 3)) + ASSERT_OK_AND_ASSIGN(auto schema_name, MakeArrayOfNull(utf8(), 3)) + + const auto table_name = + ArrayFromJSON(utf8(), R"(["foreignTable", "intTable", "sqlite_sequence"])"); + const auto table_type = ArrayFromJSON(utf8(), R"(["table", "table", "table"])"); + + const std::shared_ptr
& expected_table = Table::Make( + SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); + + AssertTablesEqual(*expected_table, *table); +} + +TEST_F(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { + FlightCallOptions options = {}; + std::string* catalog = nullptr; + std::string* schema_filter_pattern = nullptr; + std::string table_filter_pattern = "int%"; + bool include_schema = false; + std::vector* table_types = nullptr; + + ASSERT_OK_AND_ASSIGN( + auto flight_info, + sql_client->GetTables(options, catalog, schema_filter_pattern, + &table_filter_pattern, include_schema, table_types)); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const auto catalog_name = ArrayFromJSON(utf8(), R"([null])"); + const auto schema_name = ArrayFromJSON(utf8(), R"([null])"); + const auto table_name = ArrayFromJSON(utf8(), R"(["intTable"])"); + const auto table_type = ArrayFromJSON(utf8(), R"(["table"])"); + + const std::shared_ptr
& expected_table = Table::Make( + SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); + + AssertTablesEqual(*expected_table, *table); +} + +TEST_F(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { + FlightCallOptions options = {}; + std::string* catalog = nullptr; + std::string* schema_filter_pattern = nullptr; + std::string* table_filter_pattern = nullptr; + bool include_schema = false; + std::vector table_types{"index"}; + + ASSERT_OK_AND_ASSIGN( + auto flight_info, + sql_client->GetTables(options, catalog, schema_filter_pattern, table_filter_pattern, + include_schema, &table_types)); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + AssertSchemaEqual(SqlSchema::GetTablesSchema(), table->schema()); + + ASSERT_EQ(table->num_rows(), 0); +} + +TEST_F(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { + FlightCallOptions options = {}; + std::string* catalog = nullptr; + std::string* schema_filter_pattern = nullptr; + std::string* table_filter_pattern = nullptr; + bool include_schema = false; + std::vector table_types{"table"}; + + ASSERT_OK_AND_ASSIGN( + auto flight_info, + sql_client->GetTables(options, catalog, schema_filter_pattern, table_filter_pattern, + include_schema, &table_types)); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const auto catalog_name = ArrayFromJSON(utf8(), R"([null, null, null])"); + const auto schema_name = ArrayFromJSON(utf8(), R"([null, null, null])"); + const auto table_name = + ArrayFromJSON(utf8(), R"(["foreignTable", "intTable", "sqlite_sequence"])"); + const auto table_type = ArrayFromJSON(utf8(), R"(["table", "table", "table"])"); + + const std::shared_ptr
& expected_table = Table::Make( + SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); + + AssertTablesEqual(*expected_table, *table); +} + +TEST_F(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { + FlightCallOptions options = {}; + std::string* catalog = nullptr; + std::string* schema_filter_pattern = nullptr; + std::string table_filter_pattern = "int%"; + bool include_schema = true; + std::vector* table_types = nullptr; + + ASSERT_OK_AND_ASSIGN( + auto flight_info, + sql_client->GetTables(options, catalog, schema_filter_pattern, + &table_filter_pattern, include_schema, table_types)); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const auto catalog_name = ArrayFromJSON(utf8(), R"([null])"); + const auto schema_name = ArrayFromJSON(utf8(), R"([null])"); + const auto table_name = ArrayFromJSON(utf8(), R"(["intTable"])"); + const auto table_type = ArrayFromJSON(utf8(), R"(["table"])"); + + const std::shared_ptr schema_table = arrow::schema( + {arrow::field("id", int64(), true), arrow::field("keyName", utf8(), true), + arrow::field("value", int64(), true), arrow::field("foreignId", int64(), true)}); + + ASSERT_OK_AND_ASSIGN(auto schema_buffer, ipc::SerializeSchema(*schema_table)); + + std::shared_ptr table_schema; + ArrayFromVector({schema_buffer->ToString()}, &table_schema); + + const std::shared_ptr
& expected_table = + Table::Make(SqlSchema::GetTablesSchemaWithIncludedSchema(), + {catalog_name, schema_name, table_name, table_type, table_schema}); + + AssertTablesEqual(*expected_table, *table); +} + +TEST_F(TestFlightSqlServer, TestCommandGetCatalogs) { + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetCatalogs({})); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const std::shared_ptr& expected_schema = SqlSchema::GetCatalogsSchema(); + + AssertSchemaEqual(expected_schema, table->schema()); + ASSERT_EQ(0, table->num_rows()); +} + +TEST_F(TestFlightSqlServer, TestCommandGetDbSchemas) { + FlightCallOptions options = {}; + std::string* catalog = nullptr; + std::string* schema_filter_pattern = nullptr; + ASSERT_OK_AND_ASSIGN(auto flight_info, + sql_client->GetDbSchemas(options, catalog, schema_filter_pattern)); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const std::shared_ptr& expected_schema = SqlSchema::GetDbSchemasSchema(); + + AssertSchemaEqual(expected_schema, table->schema()); + ASSERT_EQ(0, table->num_rows()); +} + +TEST_F(TestFlightSqlServer, TestCommandGetTableTypes) { + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetTableTypes({})); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const auto table_type = ArrayFromJSON(utf8(), R"(["table"])"); + + const std::shared_ptr
& expected_table = + Table::Make(SqlSchema::GetTableTypesSchema(), {table_type}); + AssertTablesEqual(*expected_table, *table); +} + +TEST_F(TestFlightSqlServer, TestCommandStatementUpdate) { + int64_t result; + ASSERT_OK_AND_ASSIGN(result, + sql_client->ExecuteUpdate( + {}, + "INSERT INTO intTable (keyName, value) VALUES " + "('KEYNAME1', 1001), ('KEYNAME2', 1002), ('KEYNAME3', 1003)")); + ASSERT_EQ(3, result); + + ASSERT_OK_AND_ASSIGN(result, sql_client->ExecuteUpdate( + {}, + "UPDATE intTable SET keyName = 'KEYNAME1' " + "WHERE keyName = 'KEYNAME2' OR keyName = 'KEYNAME3'")); + ASSERT_EQ(2, result); + + ASSERT_OK_AND_ASSIGN( + result, + sql_client->ExecuteUpdate({}, "DELETE FROM intTable WHERE keyName = 'KEYNAME1'")); + ASSERT_EQ(3, result); +} + +TEST_F(TestFlightSqlServer, TestCommandPreparedStatementQuery) { + ASSERT_OK_AND_ASSIGN(auto prepared_statement, + sql_client->Prepare({}, "SELECT * FROM intTable")); + + ASSERT_OK_AND_ASSIGN(auto flight_info, prepared_statement->Execute()); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const std::shared_ptr& expected_schema = + arrow::schema({arrow::field("id", int64()), arrow::field("keyName", utf8()), + arrow::field("value", int64()), arrow::field("foreignId", int64())}); + + const auto id_array = ArrayFromJSON(int64(), R"([1, 2, 3, 4])"); + const auto keyname_array = + ArrayFromJSON(utf8(), R"(["one", "zero", "negative one", null])"); + const auto value_array = ArrayFromJSON(int64(), R"([1, 0, -1, null])"); + const auto foreignId_array = ArrayFromJSON(int64(), R"([1, 1, 1, null])"); + + const std::shared_ptr
& expected_table = Table::Make( + expected_schema, {id_array, keyname_array, value_array, foreignId_array}); + + AssertTablesEqual(*expected_table, *table); +} + +TEST_F(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) { + ASSERT_OK_AND_ASSIGN( + auto prepared_statement, + sql_client->Prepare({}, "SELECT * FROM intTable WHERE keyName LIKE ?")); + + auto parameter_schema = prepared_statement->parameter_schema(); + + const std::shared_ptr& expected_parameter_schema = + arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); + + AssertSchemaEqual(expected_parameter_schema, parameter_schema); + + std::shared_ptr type_ids = ArrayFromJSON(int8(), R"([0])"); + std::shared_ptr offsets = ArrayFromJSON(int32(), R"([0])"); + std::shared_ptr string_array = ArrayFromJSON(utf8(), R"(["%one"])"); + std::shared_ptr bytes_array = ArrayFromJSON(binary(), R"([])"); + std::shared_ptr bigint_array = ArrayFromJSON(int64(), R"([])"); + std::shared_ptr double_array = ArrayFromJSON(float64(), R"([])"); + + ASSERT_OK_AND_ASSIGN( + auto parameter_1_array, + DenseUnionArray::Make(*type_ids, *offsets, + {string_array, bytes_array, bigint_array, double_array}, + {"string", "bytes", "bigint", "double"}, {0, 1, 2, 3})); + + const std::shared_ptr& record_batch = + RecordBatch::Make(parameter_schema, 1, {parameter_1_array}); + + ASSERT_OK(prepared_statement->SetParameters(record_batch)); + + ASSERT_OK_AND_ASSIGN(auto flight_info, prepared_statement->Execute()); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const std::shared_ptr& expected_schema = + arrow::schema({arrow::field("id", int64()), arrow::field("keyName", utf8()), + arrow::field("value", int64()), arrow::field("foreignId", int64())}); + + const auto id_array = ArrayFromJSON(int64(), R"([1, 3])"); + const auto keyname_array = ArrayFromJSON(utf8(), R"(["one", "negative one"])"); + const auto value_array = ArrayFromJSON(int64(), R"([1, -1])"); + const auto foreignId_array = ArrayFromJSON(int64(), R"([1, 1])"); + + const std::shared_ptr
& expected_table = Table::Make( + expected_schema, {id_array, keyname_array, value_array, foreignId_array}); + + AssertTablesEqual(*expected_table, *table); +} + +TEST_F(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding) { + ASSERT_OK_AND_ASSIGN( + auto prepared_statement, + sql_client->Prepare( + {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', ?)")); + + auto parameter_schema = prepared_statement->parameter_schema(); + + const std::shared_ptr& expected_parameter_schema = + arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); + + AssertSchemaEqual(expected_parameter_schema, parameter_schema); + + std::shared_ptr type_ids = ArrayFromJSON(int8(), R"([2])"); + std::shared_ptr offsets = ArrayFromJSON(int32(), R"([0])"); + std::shared_ptr string_array = ArrayFromJSON(utf8(), R"([])"); + std::shared_ptr bytes_array = ArrayFromJSON(binary(), R"([])"); + std::shared_ptr bigint_array = ArrayFromJSON(int64(), R"([999])"); + std::shared_ptr double_array = ArrayFromJSON(float64(), R"([])"); + + ASSERT_OK_AND_ASSIGN( + auto parameter_1_array, + DenseUnionArray::Make(*type_ids, *offsets, + {string_array, bytes_array, bigint_array, double_array}, + {"string", "bytes", "bigint", "double"}, {0, 1, 2, 3})); + + const std::shared_ptr& record_batch = + RecordBatch::Make(parameter_schema, 1, {parameter_1_array}); + + ASSERT_OK(prepared_statement->SetParameters(record_batch)); + + ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); + + ASSERT_OK_AND_EQ(1, prepared_statement->ExecuteUpdate()); + + ASSERT_OK_AND_EQ(5, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); + + ASSERT_OK_AND_EQ(1, sql_client->ExecuteUpdate( + {}, "DELETE FROM intTable WHERE keyName = 'new_value'")); + + ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); +} + +TEST_F(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { + ASSERT_OK_AND_ASSIGN( + auto prepared_statement, + sql_client->Prepare( + {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', 999)")); + + ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); + + ASSERT_OK_AND_EQ(1, prepared_statement->ExecuteUpdate()); + + ASSERT_OK_AND_EQ(5, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); + + ASSERT_OK_AND_EQ(1, sql_client->ExecuteUpdate( + {}, "DELETE FROM intTable WHERE keyName = 'new_value'")); + + ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); +} + +TEST_F(TestFlightSqlServer, TestCommandGetPrimaryKeys) { + FlightCallOptions options = {}; + TableRef table_ref = {util::nullopt, util::nullopt, "int%"}; + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetPrimaryKeys(options, table_ref)); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const auto catalog_name = ArrayFromJSON(utf8(), R"([null])"); + const auto schema_name = ArrayFromJSON(utf8(), R"([null])"); + const auto key_name = ArrayFromJSON(utf8(), R"([null])"); + const auto table_name = ArrayFromJSON(utf8(), R"(["intTable"])"); + const auto column_name = ArrayFromJSON(utf8(), R"(["id"])"); + const auto key_sequence = ArrayFromJSON(int64(), R"([1])"); + + const std::shared_ptr
& expected_table = Table::Make( + SqlSchema::GetPrimaryKeysSchema(), + {catalog_name, schema_name, table_name, column_name, key_sequence, key_name}); + + AssertTablesEqual(*expected_table, *table); +} + +TEST_F(TestFlightSqlServer, TestCommandGetImportedKeys) { + FlightCallOptions options = {}; + TableRef table_ref = {util::nullopt, util::nullopt, "intTable"}; + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetImportedKeys(options, table_ref)); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const auto pk_catalog_name = ArrayFromJSON(utf8(), R"([null])"); + const auto pk_schema_name = ArrayFromJSON(utf8(), R"([null])"); + const auto pk_table_name = ArrayFromJSON(utf8(), R"(["foreignTable"])"); + const auto pk_column_name = ArrayFromJSON(utf8(), R"(["id"])"); + const auto fk_catalog_name = ArrayFromJSON(utf8(), R"([null])"); + const auto fk_schema_name = ArrayFromJSON(utf8(), R"([null])"); + const auto fk_table_name = ArrayFromJSON(utf8(), R"(["intTable"])"); + const auto fk_column_name = ArrayFromJSON(utf8(), R"(["foreignId"])"); + const auto key_sequence = ArrayFromJSON(int32(), R"([0])"); + const auto fk_key_name = ArrayFromJSON(utf8(), R"([null])"); + const auto pk_key_name = ArrayFromJSON(utf8(), R"([null])"); + const auto update_rule = ArrayFromJSON(uint8(), R"([3])"); + const auto delete_rule = ArrayFromJSON(uint8(), R"([3])"); + + const std::shared_ptr
& expected_table = + Table::Make(SqlSchema::GetImportedKeysSchema(), + {pk_catalog_name, pk_schema_name, pk_table_name, pk_column_name, + fk_catalog_name, fk_schema_name, fk_table_name, fk_column_name, + key_sequence, fk_key_name, pk_key_name, update_rule, delete_rule}); + AssertTablesEqual(*expected_table, *table); +} + +TEST_F(TestFlightSqlServer, TestCommandGetExportedKeys) { + FlightCallOptions options = {}; + TableRef table_ref = {util::nullopt, util::nullopt, "foreignTable"}; + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetExportedKeys(options, table_ref)); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const auto pk_catalog_name = ArrayFromJSON(utf8(), R"([null])"); + const auto pk_schema_name = ArrayFromJSON(utf8(), R"([null])"); + const auto pk_table_name = ArrayFromJSON(utf8(), R"(["foreignTable"])"); + const auto pk_column_name = ArrayFromJSON(utf8(), R"(["id"])"); + const auto fk_catalog_name = ArrayFromJSON(utf8(), R"([null])"); + const auto fk_schema_name = ArrayFromJSON(utf8(), R"([null])"); + const auto fk_table_name = ArrayFromJSON(utf8(), R"(["intTable"])"); + const auto fk_column_name = ArrayFromJSON(utf8(), R"(["foreignId"])"); + const auto key_sequence = ArrayFromJSON(int32(), R"([0])"); + const auto fk_key_name = ArrayFromJSON(utf8(), R"([null])"); + const auto pk_key_name = ArrayFromJSON(utf8(), R"([null])"); + const auto update_rule = ArrayFromJSON(uint8(), R"([3])"); + const auto delete_rule = ArrayFromJSON(uint8(), R"([3])"); + + const std::shared_ptr
& expected_table = + Table::Make(SqlSchema::GetExportedKeysSchema(), + {pk_catalog_name, pk_schema_name, pk_table_name, pk_column_name, + fk_catalog_name, fk_schema_name, fk_table_name, fk_column_name, + key_sequence, fk_key_name, pk_key_name, update_rule, delete_rule}); + AssertTablesEqual(*expected_table, *table); +} + +TEST_F(TestFlightSqlServer, TestCommandGetCrossReference) { + FlightCallOptions options = {}; + TableRef pk_table_ref = {util::nullopt, util::nullopt, "foreignTable"}; + TableRef fk_table_ref = {util::nullopt, util::nullopt, "intTable"}; + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetCrossReference( + options, pk_table_ref, fk_table_ref)); + + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const auto pk_catalog_name = ArrayFromJSON(utf8(), R"([null])"); + const auto pk_schema_name = ArrayFromJSON(utf8(), R"([null])"); + const auto pk_table_name = ArrayFromJSON(utf8(), R"(["foreignTable"])"); + const auto pk_column_name = ArrayFromJSON(utf8(), R"(["id"])"); + const auto fk_catalog_name = ArrayFromJSON(utf8(), R"([null])"); + const auto fk_schema_name = ArrayFromJSON(utf8(), R"([null])"); + const auto fk_table_name = ArrayFromJSON(utf8(), R"(["intTable"])"); + const auto fk_column_name = ArrayFromJSON(utf8(), R"(["foreignId"])"); + const auto key_sequence = ArrayFromJSON(int32(), R"([0])"); + const auto fk_key_name = ArrayFromJSON(utf8(), R"([null])"); + const auto pk_key_name = ArrayFromJSON(utf8(), R"([null])"); + const auto update_rule = ArrayFromJSON(uint8(), R"([3])"); + const auto delete_rule = ArrayFromJSON(uint8(), R"([3])"); + + const std::shared_ptr
& expected_table = + Table::Make(SqlSchema::GetCrossReferenceSchema(), + {pk_catalog_name, pk_schema_name, pk_table_name, pk_column_name, + fk_catalog_name, fk_schema_name, fk_table_name, fk_column_name, + key_sequence, fk_key_name, pk_key_name, update_rule, delete_rule}); + AssertTablesEqual(*expected_table, *table); +} + +TEST_F(TestFlightSqlServer, TestCommandGetSqlInfo) { + const auto& sql_info_expected_results = sql::example::GetSqlInfoResultMap(); + std::vector sql_info_ids; + sql_info_ids.reserve(sql_info_expected_results.size()); + for (const auto& sql_info_expected_result : sql_info_expected_results) { + sql_info_ids.push_back(sql_info_expected_result.first); + } + + FlightCallOptions call_options; + ASSERT_OK_AND_ASSIGN(auto flight_info, + sql_client->GetSqlInfo(call_options, sql_info_ids)); + ASSERT_OK_AND_ASSIGN( + auto reader, sql_client->DoGet(call_options, flight_info->endpoints()[0].ticket)); + std::shared_ptr
results; + ASSERT_OK(reader->ReadAll(&results)); + ASSERT_EQ(2, results->num_columns()); + ASSERT_EQ(sql_info_ids.size(), results->num_rows()); + const auto& col_name = results->column(0); + const auto& col_value = results->column(1); + for (int32_t i = 0; i < col_name->num_chunks(); i++) { + const auto* col_name_chunk_data = + col_name->chunk(i)->data()->GetValuesSafe(1); + const auto& col_value_chunk = col_value->chunk(i); + for (int64_t row = 0; row < col_value->length(); row++) { + ASSERT_OK_AND_ASSIGN(const auto& scalar, col_value_chunk->GetScalar(row)); + const SqlInfoDenseUnionValidator validator( + reinterpret_cast(*scalar)); + const auto& expected_result = + sql_info_expected_results.at(col_name_chunk_data[row]); + arrow::util::visit(validator, expected_result); + } + } +} + +TEST_F(TestFlightSqlServer, TestCommandGetSqlInfoNoInfo) { + FlightCallOptions call_options; + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetSqlInfo(call_options, {999999})); + + EXPECT_RAISES_WITH_MESSAGE_THAT( + KeyError, ::testing::HasSubstr("No information for SQL info number 999999."), + sql_client->DoGet(call_options, flight_info->endpoints()[0].ticket)); +} + +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/sql_info_internal.cc b/cpp/src/arrow/flight/sql/sql_info_internal.cc new file mode 100644 index 00000000000..74718fb7cb5 --- /dev/null +++ b/cpp/src/arrow/flight/sql/sql_info_internal.cc @@ -0,0 +1,101 @@ +// 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. + +#include "arrow/flight/sql/sql_info_internal.h" + +#include "arrow/buffer.h" +#include "arrow/builder.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace internal { + +Status SqlInfoResultAppender::operator()(const std::string& value) { + ARROW_RETURN_NOT_OK(value_builder_->Append(kStringValueIndex)); + ARROW_RETURN_NOT_OK(string_value_builder_->Append(value)); + return Status::OK(); +} + +Status SqlInfoResultAppender::operator()(const bool value) { + ARROW_RETURN_NOT_OK(value_builder_->Append(kBoolValueIndex)); + ARROW_RETURN_NOT_OK(bool_value_builder_->Append(value)); + return Status::OK(); +} + +Status SqlInfoResultAppender::operator()(const int64_t value) { + ARROW_RETURN_NOT_OK(value_builder_->Append(kBigIntValueIndex)); + ARROW_RETURN_NOT_OK(bigint_value_builder_->Append(value)); + return Status::OK(); +} + +Status SqlInfoResultAppender::operator()(const int32_t value) { + ARROW_RETURN_NOT_OK(value_builder_->Append(kInt32BitMaskIndex)); + ARROW_RETURN_NOT_OK(int32_bitmask_builder_->Append(value)); + return Status::OK(); +} + +Status SqlInfoResultAppender::operator()(const std::vector& value) { + ARROW_RETURN_NOT_OK(value_builder_->Append(kStringListIndex)); + ARROW_RETURN_NOT_OK(string_list_builder_->Append()); + auto* string_list_child = + reinterpret_cast(string_list_builder_->value_builder()); + for (const auto& string : value) { + ARROW_RETURN_NOT_OK(string_list_child->Append(string)); + } + return Status::OK(); +} + +Status SqlInfoResultAppender::operator()( + const std::unordered_map>& value) { + ARROW_RETURN_NOT_OK(value_builder_->Append(kInt32ToInt32ListIndex)); + ARROW_RETURN_NOT_OK(int32_to_int32_list_builder_->Append()); + for (const auto& pair : value) { + ARROW_RETURN_NOT_OK( + reinterpret_cast(int32_to_int32_list_builder_->key_builder()) + ->Append(pair.first)); + auto* int32_list_builder = + reinterpret_cast(int32_to_int32_list_builder_->item_builder()); + ARROW_RETURN_NOT_OK(int32_list_builder->Append()); + auto* int32_list_child = + reinterpret_cast(int32_list_builder->value_builder()); + for (const auto& int32 : pair.second) { + ARROW_RETURN_NOT_OK(int32_list_child->Append(int32)); + } + } + return Status::OK(); +} + +SqlInfoResultAppender::SqlInfoResultAppender(DenseUnionBuilder* value_builder) + : value_builder_(value_builder), + string_value_builder_( + reinterpret_cast(value_builder_->child(kStringValueIndex))), + bool_value_builder_( + reinterpret_cast(value_builder_->child(kBoolValueIndex))), + bigint_value_builder_( + reinterpret_cast(value_builder_->child(kBigIntValueIndex))), + int32_bitmask_builder_( + reinterpret_cast(value_builder_->child(kInt32BitMaskIndex))), + string_list_builder_( + reinterpret_cast(value_builder_->child(kStringListIndex))), + int32_to_int32_list_builder_( + reinterpret_cast(value_builder_->child(kInt32ToInt32ListIndex))) {} + +} // namespace internal +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/sql_info_internal.h b/cpp/src/arrow/flight/sql/sql_info_internal.h new file mode 100644 index 00000000000..b18789c2549 --- /dev/null +++ b/cpp/src/arrow/flight/sql/sql_info_internal.h @@ -0,0 +1,87 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include "arrow/flight/sql/types.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace internal { + +/// \brief Auxiliary class used to populate GetSqlInfo's DenseUnionArray with different +/// data types. +class SqlInfoResultAppender { + public: + /// \brief Append a string to the DenseUnionBuilder. + /// \param[in] value Value to be appended. + Status operator()(const std::string& value); + + /// \brief Append a bool to the DenseUnionBuilder. + /// \param[in] value Value to be appended. + Status operator()(bool value); + + /// \brief Append a int64_t to the DenseUnionBuilder. + /// \param[in] value Value to be appended. + Status operator()(int64_t value); + + /// \brief Append a int32_t to the DenseUnionBuilder. + /// \param[in] value Value to be appended. + Status operator()(int32_t value); + + /// \brief Append a string list to the DenseUnionBuilder. + /// \param[in] value Value to be appended. + Status operator()(const std::vector& value); + + /// \brief Append a int32 to int32 list map to the DenseUnionBuilder. + /// \param[in] value Value to be appended. + Status operator()(const std::unordered_map>& value); + + /// \brief Create a Variant visitor that appends data to given + /// DenseUnionBuilder. \param[in] value_builder DenseUnionBuilder to append data to. + explicit SqlInfoResultAppender(DenseUnionBuilder* value_builder); + + SqlInfoResultAppender(const SqlInfoResultAppender&) = delete; + SqlInfoResultAppender(SqlInfoResultAppender&&) = delete; + SqlInfoResultAppender& operator=(const SqlInfoResultAppender&) = delete; + + private: + DenseUnionBuilder* value_builder_; + + // Builders for each child on dense union + StringBuilder* string_value_builder_; + BooleanBuilder* bool_value_builder_; + Int64Builder* bigint_value_builder_; + Int32Builder* int32_bitmask_builder_; + ListBuilder* string_list_builder_; + MapBuilder* int32_to_int32_list_builder_; + + enum : int8_t { + kStringValueIndex = 0, + kBoolValueIndex = 1, + kBigIntValueIndex = 2, + kInt32BitMaskIndex = 3, + kStringListIndex = 4, + kInt32ToInt32ListIndex = 5 + }; +}; + +} // namespace internal +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/sql/test_app_cli.cc b/cpp/src/arrow/flight/sql/test_app_cli.cc new file mode 100644 index 00000000000..43c37bee2fe --- /dev/null +++ b/cpp/src/arrow/flight/sql/test_app_cli.cc @@ -0,0 +1,197 @@ +// 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. + +#include + +#include +#include +#include + +#include "arrow/array/builder_binary.h" +#include "arrow/array/builder_primitive.h" +#include "arrow/flight/api.h" +#include "arrow/flight/sql/api.h" +#include "arrow/io/memory.h" +#include "arrow/pretty_print.h" +#include "arrow/status.h" +#include "arrow/table.h" +#include "arrow/util/optional.h" + +using arrow::Result; +using arrow::Schema; +using arrow::Status; +using arrow::flight::ClientAuthHandler; +using arrow::flight::FlightCallOptions; +using arrow::flight::FlightClient; +using arrow::flight::FlightDescriptor; +using arrow::flight::FlightEndpoint; +using arrow::flight::FlightInfo; +using arrow::flight::FlightStreamChunk; +using arrow::flight::FlightStreamReader; +using arrow::flight::Location; +using arrow::flight::Ticket; +using arrow::flight::sql::FlightSqlClient; +using arrow::flight::sql::TableRef; + +DEFINE_string(host, "localhost", "Host to connect to"); +DEFINE_int32(port, 32010, "Port to connect to"); +DEFINE_string(username, "", "Username"); +DEFINE_string(password, "", "Password"); + +DEFINE_string(command, "", "Method to run"); +DEFINE_string(query, "", "Query"); +DEFINE_string(catalog, "", "Catalog"); +DEFINE_string(schema, "", "Schema"); +DEFINE_string(table, "", "Table"); + +Status PrintResultsForEndpoint(FlightSqlClient& client, + const FlightCallOptions& call_options, + const FlightEndpoint& endpoint) { + ARROW_ASSIGN_OR_RAISE(auto stream, client.DoGet(call_options, endpoint.ticket)); + + const arrow::Result>& schema = stream->GetSchema(); + ARROW_RETURN_NOT_OK(schema); + + std::cout << "Schema:" << std::endl; + std::cout << schema->get()->ToString() << std::endl << std::endl; + + std::cout << "Results:" << std::endl; + + FlightStreamChunk chunk; + int64_t num_rows = 0; + + while (true) { + ARROW_RETURN_NOT_OK(stream->Next(&chunk)); + if (chunk.data == nullptr) { + break; + } + std::cout << chunk.data->ToString() << std::endl; + num_rows += chunk.data->num_rows(); + } + + std::cout << "Total: " << num_rows << std::endl; + + return Status::OK(); +} + +Status PrintResults(FlightSqlClient& client, const FlightCallOptions& call_options, + const std::unique_ptr& info) { + const std::vector& endpoints = info->endpoints(); + + for (size_t i = 0; i < endpoints.size(); i++) { + std::cout << "Results from endpoint " << i + 1 << " of " << endpoints.size() + << std::endl; + ARROW_RETURN_NOT_OK(PrintResultsForEndpoint(client, call_options, endpoints[i])); + } + + return Status::OK(); +} + +Status RunMain() { + std::unique_ptr client; + Location location; + ARROW_RETURN_NOT_OK(Location::ForGrpcTcp(FLAGS_host, FLAGS_port, &location)); + ARROW_RETURN_NOT_OK(FlightClient::Connect(location, &client)); + + FlightCallOptions call_options; + + if (!FLAGS_username.empty() || !FLAGS_password.empty()) { + Result> bearer_result = + client->AuthenticateBasicToken({}, FLAGS_username, FLAGS_password); + ARROW_RETURN_NOT_OK(bearer_result); + + call_options.headers.push_back(bearer_result.ValueOrDie()); + } + + FlightSqlClient sql_client(std::move(client)); + + if (FLAGS_command == "ExecuteUpdate") { + ARROW_ASSIGN_OR_RAISE(auto rows, sql_client.ExecuteUpdate(call_options, FLAGS_query)); + + std::cout << "Result: " << rows << std::endl; + + return Status::OK(); + } + + std::unique_ptr info; + + if (FLAGS_command == "Execute") { + ARROW_ASSIGN_OR_RAISE(info, sql_client.Execute(call_options, FLAGS_query)); + } else if (FLAGS_command == "GetCatalogs") { + ARROW_ASSIGN_OR_RAISE(info, sql_client.GetCatalogs(call_options)); + } else if (FLAGS_command == "PreparedStatementExecute") { + ARROW_ASSIGN_OR_RAISE(auto prepared_statement, + sql_client.Prepare(call_options, FLAGS_query)); + ARROW_ASSIGN_OR_RAISE(info, prepared_statement->Execute()); + } else if (FLAGS_command == "PreparedStatementExecuteParameterBinding") { + ARROW_ASSIGN_OR_RAISE(auto prepared_statement, sql_client.Prepare({}, FLAGS_query)); + auto parameter_schema = prepared_statement->parameter_schema(); + auto result_set_schema = prepared_statement->dataset_schema(); + + std::cout << result_set_schema->ToString(false) << std::endl; + arrow::Int64Builder int_builder; + ARROW_RETURN_NOT_OK(int_builder.Append(1)); + std::shared_ptr int_array; + ARROW_RETURN_NOT_OK(int_builder.Finish(&int_array)); + std::shared_ptr result; + result = arrow::RecordBatch::Make(parameter_schema, 1, {int_array}); + + ARROW_RETURN_NOT_OK(prepared_statement->SetParameters(result)); + ARROW_ASSIGN_OR_RAISE(info, prepared_statement->Execute()); + } else if (FLAGS_command == "GetDbSchemas") { + ARROW_ASSIGN_OR_RAISE( + info, sql_client.GetDbSchemas(call_options, &FLAGS_catalog, &FLAGS_schema)); + } else if (FLAGS_command == "GetTableTypes") { + ARROW_ASSIGN_OR_RAISE(info, sql_client.GetTableTypes(call_options)); + } else if (FLAGS_command == "GetTables") { + ARROW_ASSIGN_OR_RAISE( + info, sql_client.GetTables(call_options, &FLAGS_catalog, &FLAGS_schema, + &FLAGS_table, false, nullptr)); + } else if (FLAGS_command == "GetExportedKeys") { + TableRef table_ref = {arrow::util::make_optional(FLAGS_catalog), + arrow::util::make_optional(FLAGS_schema), FLAGS_table}; + ARROW_ASSIGN_OR_RAISE(info, sql_client.GetExportedKeys(call_options, table_ref)); + } else if (FLAGS_command == "GetImportedKeys") { + TableRef table_ref = {arrow::util::make_optional(FLAGS_catalog), + arrow::util::make_optional(FLAGS_schema), FLAGS_table}; + ARROW_ASSIGN_OR_RAISE(info, sql_client.GetImportedKeys(call_options, table_ref)); + } else if (FLAGS_command == "GetPrimaryKeys") { + TableRef table_ref = {arrow::util::make_optional(FLAGS_catalog), + arrow::util::make_optional(FLAGS_schema), FLAGS_table}; + ARROW_ASSIGN_OR_RAISE(info, sql_client.GetPrimaryKeys(call_options, table_ref)); + } else if (FLAGS_command == "GetSqlInfo") { + ARROW_ASSIGN_OR_RAISE(info, sql_client.GetSqlInfo(call_options, {})); + } + + if (info != NULLPTR && + !boost::istarts_with(FLAGS_command, "PreparedStatementExecute")) { + return PrintResults(sql_client, call_options, info); + } + + return Status::OK(); +} + +int main(int argc, char** argv) { + gflags::ParseCommandLineFlags(&argc, &argv, true); + + Status st = RunMain(); + if (!st.ok()) { + std::cerr << st << std::endl; + return 1; + } + return 0; +} diff --git a/cpp/src/arrow/flight/sql/test_server_cli.cc b/cpp/src/arrow/flight/sql/test_server_cli.cc new file mode 100644 index 00000000000..8074ab534bd --- /dev/null +++ b/cpp/src/arrow/flight/sql/test_server_cli.cc @@ -0,0 +1,63 @@ +// 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. + +#include + +#include +#include +#include + +#include "arrow/flight/server.h" +#include "arrow/flight/sql/example/sqlite_server.h" +#include "arrow/flight/test_integration.h" +#include "arrow/flight/test_util.h" +#include "arrow/io/test_common.h" +#include "arrow/testing/json_integration.h" +#include "arrow/util/logging.h" + +DEFINE_int32(port, 31337, "Server port to listen on"); + +arrow::Status RunMain() { + arrow::flight::Location location; + ARROW_CHECK_OK(arrow::flight::Location::ForGrpcTcp("0.0.0.0", FLAGS_port, &location)); + arrow::flight::FlightServerOptions options(location); + + std::shared_ptr server; + ARROW_ASSIGN_OR_RAISE(server, + arrow::flight::sql::example::SQLiteFlightSqlServer::Create()) + + ARROW_CHECK_OK(server->Init(options)); + // Exit with a clean error code (0) on SIGTERM + ARROW_CHECK_OK(server->SetShutdownOnSignals({SIGTERM})); + + std::cout << "Server listening on localhost:" << server->port() << std::endl; + ARROW_CHECK_OK(server->Serve()); + + return arrow::Status::OK(); +} + +int main(int argc, char** argv) { + gflags::SetUsageMessage("Integration testing server for Flight SQL."); + gflags::ParseCommandLineFlags(&argc, &argv, true); + + arrow::Status st = RunMain(); + if (!st.ok()) { + std::cerr << st << std::endl; + return 1; + } + return 0; +} diff --git a/cpp/src/arrow/flight/sql/types.h b/cpp/src/arrow/flight/sql/types.h new file mode 100644 index 00000000000..44b8bca4718 --- /dev/null +++ b/cpp/src/arrow/flight/sql/types.h @@ -0,0 +1,890 @@ +// 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. + +#pragma once + +#include +#include +#include +#include + +#include "arrow/type_fwd.h" +#include "arrow/util/optional.h" +#include "arrow/util/variant.h" + +namespace arrow { +namespace flight { +namespace sql { + +/// \brief Variant supporting all possible types on SQL info. +using SqlInfoResult = + arrow::util::Variant, + std::unordered_map>>; + +/// \brief Map SQL info identifier to its value. +using SqlInfoResultMap = std::unordered_map; + +/// \brief Options to be set in the SqlInfo. +struct SqlInfoOptions { + 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 and schema 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_COMPARISONS, + * 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 + // procedure name. + SQL_SCHEMA_NAME_LENGTH = 552, + + // Retrieves a uint32 value representing the maximum number of bytes allowed in a + // single row. + 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, + }; + + 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 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, + }; +}; + +/// \brief Table reference, optionally containing table's catalog and db_schema. +struct TableRef { + util::optional catalog; + util::optional db_schema; + std::string table; +}; + +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/vcpkg.json b/cpp/vcpkg.json index a6e9c68b57d..d9505b67383 100644 --- a/cpp/vcpkg.json +++ b/cpp/vcpkg.json @@ -43,6 +43,7 @@ "rapidjson", "re2", "snappy", + "sqlite3", "thrift", "utf8proc", "zlib", diff --git a/docker-compose.yml b/docker-compose.yml index 9bf5bd0841d..17a90937834 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -266,6 +266,7 @@ services: ARROW_CXXFLAGS: "-Og" # Shrink test runtime by enabling minimal optimizations ARROW_ENABLE_TIMING_TESTS: # inherit ARROW_FLIGHT: "OFF" + ARROW_FLIGHT_SQL: "OFF" ARROW_GANDIVA: "OFF" ARROW_JEMALLOC: "OFF" ARROW_RUNTIME_SIMD_LEVEL: "AVX2" # AVX512 not supported by Valgrind (ARROW-9851) @@ -999,6 +1000,7 @@ services: environment: <<: *ccache ARROW_FLIGHT: "OFF" + ARROW_FLIGHT_SQL: "OFF" ARROW_GANDIVA: "OFF" volumes: *conda-volumes command: @@ -1594,6 +1596,7 @@ services: environment: <<: *ccache ARROW_FLIGHT: "OFF" + ARROW_FLIGHT_SQL: "OFF" ARROW_GANDIVA: "OFF" ARROW_PLASMA: "OFF" ARROW_HIVESERVER2: "ON" diff --git a/format/FlightSql.proto b/format/FlightSql.proto new file mode 100644 index 00000000000..23ada5c6e48 --- /dev/null +++ b/format/FlightSql.proto @@ -0,0 +1,1336 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * 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; +} diff --git a/java/flight/flight-core/pom.xml b/java/flight/flight-core/pom.xml index b1f00eb83f9..ce307e52770 100644 --- a/java/flight/flight-core/pom.xml +++ b/java/flight/flight-core/pom.xml @@ -25,7 +25,7 @@ 1.41.0 - 3.7.1 + 3.17.3 1