From 370c92ef1d9748feef8c61e2b19515a558124939 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 23 Aug 2021 15:28:53 -0300 Subject: [PATCH 001/237] WIP: Set up flight-sql project on cpp directory --- cpp/CMakeLists.txt | 4 + cpp/cmake_modules/DefineOptions.cmake | 2 + cpp/cmake_modules/FindArrowFlightSql.cmake | 89 ++++++++++ cpp/src/arrow/CMakeLists.txt | 4 + .../flight-sql/ArrowFlightSqlConfig.cmake.in | 36 ++++ .../arrow/flight/flight-sql/CMakeLists.txt | 165 ++++++++++++++++++ .../flight/flight-sql/arrow-flight-sql.pc.in | 25 +++ cpp/src/arrow/flight/flight-sql/client.cc | 24 +++ cpp/src/arrow/flight/flight-sql/client.h | 14 ++ .../arrow/flight/flight-sql/client_test.cc | 26 +++ 10 files changed, 389 insertions(+) create mode 100644 cpp/cmake_modules/FindArrowFlightSql.cmake create mode 100644 cpp/src/arrow/flight/flight-sql/ArrowFlightSqlConfig.cmake.in create mode 100644 cpp/src/arrow/flight/flight-sql/CMakeLists.txt create mode 100644 cpp/src/arrow/flight/flight-sql/arrow-flight-sql.pc.in create mode 100644 cpp/src/arrow/flight/flight-sql/client.cc create mode 100644 cpp/src/arrow/flight/flight-sql/client.h create mode 100644 cpp/src/arrow/flight/flight-sql/client_test.cc 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..46fa8e90229 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" 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..e4a3b772fa0 --- /dev/null +++ b/cpp/cmake_modules/FindArrowFlightSql.cmake @@ -0,0 +1,89 @@ +# 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 + "${ARROW_HOME}" + arrow_flight + arrow/flight/api.h + ArrowFlight + arrow-flight) + 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( + ArrowFlight + 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 ${ArrowFlight_FOUND}) + +if(ArrowFlight_FOUND AND NOT ArrowFlight_FIND_QUIETLY) + message(STATUS "Found the Arrow Flight by ${ARROW_FLIGHT_SQL_FIND_APPROACH}") + message(STATUS "Found the Arrow Flight shared library: ${ARROW_FLIGHT_SQL_SHARED_LIB}") + message(STATUS "Found the Arrow Flight import library: ${ARROW_FLIGHT_SQL_IMPORT_LIB}") + message(STATUS "Found the Arrow Flight static library: ${ARROW_FLIGHT_SQL_STATIC_LIB}") +endif() diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 5736c557bd0..747bfaa489f 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/flight-sql) +endif() + if(ARROW_HIVESERVER2) add_subdirectory(dbi/hiveserver2) endif() diff --git a/cpp/src/arrow/flight/flight-sql/ArrowFlightSqlConfig.cmake.in b/cpp/src/arrow/flight/flight-sql/ArrowFlightSqlConfig.cmake.in new file mode 100644 index 00000000000..f203e143e0b --- /dev/null +++ b/cpp/src/arrow/flight/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_shared OR TARGET arrow_flight_static)) + include("${CMAKE_CURRENT_LIST_DIR}/ArrowFlightSqlTargets.cmake") +endif() diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt new file mode 100644 index 00000000000..e9b4d38bf5f --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -0,0 +1,165 @@ +# 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/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" + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.grpc.pb.cc" + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.grpc.pb.h") + +set(PROTO_DEPENDS ${FLIGHT_SQL_PROTO} ${ARROW_PROTOBUF_LIBPROTOBUF} gRPC::grpc_cpp_plugin) + +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} ARGS + COMMAND ${ARROW_PROTOBUF_PROTOC} "-I${FLIGHT_SQL_PROTO_PATH}" + "--grpc_out=${CMAKE_CURRENT_BINARY_DIR}" + "--plugin=protoc-gen-grpc=$" + "${FLIGHT_SQL_PROTO}") + +set_source_files_properties(${FLIGHT_SQL_GENERATED_PROTO_FILES} PROPERTIES GENERATED TRUE) + +add_custom_target(flight_sql_grpc_gen ALL DEPENDS ${FLIGHT_SQL_GENERATED_PROTO_FILES}) + +# -Werror / /WX cause try_compile to fail because there seems to be no +# way to pass -isystem $GRPC_INCLUDE_DIR instead of -I$GRPC_INCLUDE_DIR +set(CMAKE_CXX_FLAGS_BACKUP "${CMAKE_CXX_FLAGS}") +string(REPLACE "/WX" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") +string(REPLACE "-Werror " " " CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") + +# Probe the version of gRPC being used to see if it supports disabling server +# verification when using TLS. +function(test_grpc_version DST_VAR DETECT_VERSION TEST_FILE) + if(NOT DEFINED ${DST_VAR}) + message(STATUS "Checking support for TlsCredentialsOptions (gRPC >= ${DETECT_VERSION})..." + ) + get_property(CURRENT_INCLUDE_DIRECTORIES + DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} + PROPERTY INCLUDE_DIRECTORIES) + try_compile(HAS_GRPC_VERSION ${CMAKE_CURRENT_BINARY_DIR}/try_compile + SOURCES "${CMAKE_CURRENT_SOURCE_DIR}/try_compile/${TEST_FILE}" + CMAKE_FLAGS "-DINCLUDE_DIRECTORIES=${CURRENT_INCLUDE_DIRECTORIES}" + LINK_LIBRARIES gRPC::grpc++ + OUTPUT_VARIABLE TLS_CREDENTIALS_OPTIONS_CHECK_OUTPUT CXX_STANDARD 11) + if(HAS_GRPC_VERSION) + set(${DST_VAR} + "${DETECT_VERSION}" + CACHE INTERNAL "The detected (approximate) gRPC version.") + else() + message(STATUS "TlsCredentialsOptions (for gRPC ${DETECT_VERSION}) not found in grpc::experimental." + ) + message(DEBUG "Build output:") + list(APPEND CMAKE_MESSAGE_INDENT "${TEST_FILE}: ") + message(DEBUG ${TLS_CREDENTIALS_OPTIONS_CHECK_OUTPUT}) + list(REMOVE_AT CMAKE_MESSAGE_INDENT -1) + endif() + endif() +endfunction() + +# Note, we do not compile the generated Protobuf sources directly, instead +# compiling then via protocol_internal.cc which contains some gRPC template +# overrides to enable Flight-specific optimizations. See comments in +# protobuf-internal.cc +set(ARROW_FLIGHT_SQL_SRCS + client.cc) + +add_arrow_lib(arrow_flight_sql + CMAKE_PACKAGE_NAME + ArrowFlightSql + PKG_CONFIG_NAME + arrow-flight-sql + OUTPUTS + ARROW_FLIGHT_LIBRARIES + SOURCES + ${ARROW_FLIGHT_SQL_SRCS} + PRECOMPILED_HEADERS + "$<$:arrow/flight/pch.h>" + DEPENDENCIES + flight_sql_grpc_gen + SHARED_LINK_FLAGS + ${ARROW_VERSION_SCRIPT_FLAGS} # Defined in cpp/arrow/CMakeLists.txt + SHARED_LINK_LIBS + arrow_shared + ${ARROW_FLIGHT_LINK_LIBS} + STATIC_LINK_LIBS + arrow_static + ${ARROW_FLIGHT_LINK_LIBS}) + +add_arrow_test(flight_sql_test + SOURCES + client_test.cc + LABELS + "arrow_flight_sql") + +## Build test server for unit tests or benchmarks +#if(ARROW_BUILD_TESTS OR ARROW_BUILD_BENCHMARKS) +# add_executable(flight-test-server test_server.cc) +# target_link_libraries(flight-test-server ${ARROW_FLIGHT_TEST_LINK_LIBS} +# ${GFLAGS_LIBRARIES} GTest::gtest) +# +# if(ARROW_BUILD_TESTS) +# add_dependencies(arrow-flight-test flight-test-server) +# endif() +# +# add_dependencies(arrow_flight flight-test-server) +#endif() +# +#if(ARROW_BUILD_INTEGRATION) +# add_executable(flight-test-integration-server test_integration_server.cc) +# target_link_libraries(flight-test-integration-server ${ARROW_FLIGHT_TEST_LINK_LIBS} +# ${GFLAGS_LIBRARIES} GTest::gtest) +# +# add_executable(flight-test-integration-client test_integration_client.cc) +# target_link_libraries(flight-test-integration-client ${ARROW_FLIGHT_TEST_LINK_LIBS} +# ${GFLAGS_LIBRARIES} GTest::gtest) +# +# add_dependencies(arrow_flight flight-test-integration-client +# flight-test-integration-server) +# add_dependencies(arrow-integration flight-test-integration-client +# flight-test-integration-server) +#endif() +# +#if(ARROW_BUILD_BENCHMARKS) +# # Perf server for benchmarks +# set(PERF_PROTO_GENERATED_FILES "${CMAKE_CURRENT_BINARY_DIR}/perf.pb.cc" +# "${CMAKE_CURRENT_BINARY_DIR}/perf.pb.h") +# +# add_custom_command(OUTPUT ${PERF_PROTO_GENERATED_FILES} +# COMMAND ${ARROW_PROTOBUF_PROTOC} "-I${CMAKE_CURRENT_SOURCE_DIR}" +# "--cpp_out=${CMAKE_CURRENT_BINARY_DIR}" "perf.proto" +# DEPENDS ${PROTO_DEPENDS}) +# +# add_executable(arrow-flight-perf-server perf_server.cc perf.pb.cc) +# target_link_libraries(arrow-flight-perf-server ${ARROW_FLIGHT_TEST_LINK_LIBS} +# ${GFLAGS_LIBRARIES} GTest::gtest) +# +# add_executable(arrow-flight-benchmark flight_benchmark.cc perf.pb.cc) +# target_link_libraries(arrow-flight-benchmark ${ARROW_FLIGHT_TEST_LINK_LIBS} +# ${GFLAGS_LIBRARIES} GTest::gtest) +# +# add_dependencies(arrow-flight-benchmark arrow-flight-perf-server) +# +# add_dependencies(arrow_flight arrow-flight-benchmark) +#endif(ARROW_BUILD_BENCHMARKS) diff --git a/cpp/src/arrow/flight/flight-sql/arrow-flight-sql.pc.in b/cpp/src/arrow/flight/flight-sql/arrow-flight-sql.pc.in new file mode 100644 index 00000000000..157aa766ede --- /dev/null +++ b/cpp/src/arrow/flight/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 +Version: @ARROW_VERSION@ +Requires: arrow-flight +Libs: -L${libdir} -larrow_flight_sql diff --git a/cpp/src/arrow/flight/flight-sql/client.cc b/cpp/src/arrow/flight/flight-sql/client.cc new file mode 100644 index 00000000000..e4296c3e92e --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/client.cc @@ -0,0 +1,24 @@ +// 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/flight-sql/client.h" +#include "arrow/flight/flight-sql/FlightSql.pb.h" + +void x() { + arrow::flight::protocol::sql::CommandStatementQuery command; + command.set_query(""); +} diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h new file mode 100644 index 00000000000..ff72c7bde99 --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -0,0 +1,14 @@ +// +// Created by rafael on 8/23/21. +// + +#ifndef ARROW_CLIENT_H +#define ARROW_CLIENT_H + + +class client { + +}; + + +#endif //ARROW_CLIENT_H diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc new file mode 100644 index 00000000000..0dd029becb4 --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -0,0 +1,26 @@ +// 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 + +namespace arrow { + +TEST(TestFlightSql, Test) { + ASSERT_EQ(1, 1); +} + +} \ No newline at end of file From 7dc836e789b390297e191471e4aa142386bf793b Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 24 Aug 2021 13:49:32 -0300 Subject: [PATCH 002/237] Update FindArrowFlightSql.cmake --- cpp/cmake_modules/DefineOptions.cmake | 2 +- cpp/cmake_modules/FindArrowFlightSql.cmake | 24 +++++++++++----------- cpp/src/arrow/flight/flight-sql/api.h | 20 ++++++++++++++++++ 3 files changed, 33 insertions(+), 13 deletions(-) create mode 100644 cpp/src/arrow/flight/flight-sql/api.h diff --git a/cpp/cmake_modules/DefineOptions.cmake b/cpp/cmake_modules/DefineOptions.cmake index 46fa8e90229..2afbdab4a40 100644 --- a/cpp/cmake_modules/DefineOptions.cmake +++ b/cpp/cmake_modules/DefineOptions.cmake @@ -226,7 +226,7 @@ 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" OFF) + define_option(ARROW_FLIGHT_SQL "Build the Arrow Flight SQL extension" OFF) define_option(ARROW_GANDIVA "Build the Gandiva libraries" OFF) diff --git a/cpp/cmake_modules/FindArrowFlightSql.cmake b/cpp/cmake_modules/FindArrowFlightSql.cmake index e4a3b772fa0..49471dd5a2e 100644 --- a/cpp/cmake_modules/FindArrowFlightSql.cmake +++ b/cpp/cmake_modules/FindArrowFlightSql.cmake @@ -48,12 +48,12 @@ endif() find_package(Arrow ${find_package_arguments}) if(ARROW_FOUND) - arrow_find_package(ARROW_FLIGHT + arrow_find_package(ARROW_FLIGHT_SQL "${ARROW_HOME}" - arrow_flight - arrow/flight/api.h - ArrowFlight - arrow-flight) + arrow_flight_sql + arrow/flight/flight-sql/api.h + ArrowFlightSql + arrow-flight-sql) if(NOT ARROW_FLIGHT_SQL_VERSION) set(ARROW_FLIGHT_SQL_VERSION "${ARROW_VERSION}") endif() @@ -76,14 +76,14 @@ mark_as_advanced(ARROW_FLIGHT_SQL_IMPORT_LIB ARROW_FLIGHT_SQL_VERSION_MATCH) find_package_handle_standard_args( - ArrowFlight + 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 ${ArrowFlight_FOUND}) +set(ARROW_FLIGHT_SQL_FOUND ${ArrowFlightSql_FOUND}) -if(ArrowFlight_FOUND AND NOT ArrowFlight_FIND_QUIETLY) - message(STATUS "Found the Arrow Flight by ${ARROW_FLIGHT_SQL_FIND_APPROACH}") - message(STATUS "Found the Arrow Flight shared library: ${ARROW_FLIGHT_SQL_SHARED_LIB}") - message(STATUS "Found the Arrow Flight import library: ${ARROW_FLIGHT_SQL_IMPORT_LIB}") - message(STATUS "Found the Arrow Flight static library: ${ARROW_FLIGHT_SQL_STATIC_LIB}") +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/src/arrow/flight/flight-sql/api.h b/cpp/src/arrow/flight/flight-sql/api.h new file mode 100644 index 00000000000..7c2d7f6ead3 --- /dev/null +++ b/cpp/src/arrow/flight/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/flight-sql/client.h" From 2ed7b0efae2a6128f13920ed4e18d6d7d7f0d803 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 24 Aug 2021 14:12:37 -0300 Subject: [PATCH 003/237] WIP: Clean up changes --- .../flight-sql/ArrowFlightSqlConfig.cmake.in | 2 +- .../arrow/flight/flight-sql/CMakeLists.txt | 86 +------------------ .../flight/flight-sql/arrow-flight-sql.pc.in | 2 +- cpp/src/arrow/flight/flight-sql/client.cc | 5 +- cpp/src/arrow/flight/flight-sql/client.h | 21 ++++- 5 files changed, 21 insertions(+), 95 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/ArrowFlightSqlConfig.cmake.in b/cpp/src/arrow/flight/flight-sql/ArrowFlightSqlConfig.cmake.in index f203e143e0b..1658f44f418 100644 --- a/cpp/src/arrow/flight/flight-sql/ArrowFlightSqlConfig.cmake.in +++ b/cpp/src/arrow/flight/flight-sql/ArrowFlightSqlConfig.cmake.in @@ -31,6 +31,6 @@ find_dependency(ArrowFlight) # Load targets only once. If we load targets multiple times, CMake reports # already existent target error. -if(NOT (TARGET arrow_flight_shared OR TARGET arrow_flight_static)) +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/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index e9b4d38bf5f..407aa3dc056 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -49,39 +49,6 @@ set(CMAKE_CXX_FLAGS_BACKUP "${CMAKE_CXX_FLAGS}") string(REPLACE "/WX" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") string(REPLACE "-Werror " " " CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") -# Probe the version of gRPC being used to see if it supports disabling server -# verification when using TLS. -function(test_grpc_version DST_VAR DETECT_VERSION TEST_FILE) - if(NOT DEFINED ${DST_VAR}) - message(STATUS "Checking support for TlsCredentialsOptions (gRPC >= ${DETECT_VERSION})..." - ) - get_property(CURRENT_INCLUDE_DIRECTORIES - DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} - PROPERTY INCLUDE_DIRECTORIES) - try_compile(HAS_GRPC_VERSION ${CMAKE_CURRENT_BINARY_DIR}/try_compile - SOURCES "${CMAKE_CURRENT_SOURCE_DIR}/try_compile/${TEST_FILE}" - CMAKE_FLAGS "-DINCLUDE_DIRECTORIES=${CURRENT_INCLUDE_DIRECTORIES}" - LINK_LIBRARIES gRPC::grpc++ - OUTPUT_VARIABLE TLS_CREDENTIALS_OPTIONS_CHECK_OUTPUT CXX_STANDARD 11) - if(HAS_GRPC_VERSION) - set(${DST_VAR} - "${DETECT_VERSION}" - CACHE INTERNAL "The detected (approximate) gRPC version.") - else() - message(STATUS "TlsCredentialsOptions (for gRPC ${DETECT_VERSION}) not found in grpc::experimental." - ) - message(DEBUG "Build output:") - list(APPEND CMAKE_MESSAGE_INDENT "${TEST_FILE}: ") - message(DEBUG ${TLS_CREDENTIALS_OPTIONS_CHECK_OUTPUT}) - list(REMOVE_AT CMAKE_MESSAGE_INDENT -1) - endif() - endif() -endfunction() - -# Note, we do not compile the generated Protobuf sources directly, instead -# compiling then via protocol_internal.cc which contains some gRPC template -# overrides to enable Flight-specific optimizations. See comments in -# protobuf-internal.cc set(ARROW_FLIGHT_SQL_SRCS client.cc) @@ -91,7 +58,7 @@ add_arrow_lib(arrow_flight_sql PKG_CONFIG_NAME arrow-flight-sql OUTPUTS - ARROW_FLIGHT_LIBRARIES + ARROW_FLIGHT_SQL_LIBRARIES SOURCES ${ARROW_FLIGHT_SQL_SRCS} PRECOMPILED_HEADERS @@ -112,54 +79,3 @@ add_arrow_test(flight_sql_test client_test.cc LABELS "arrow_flight_sql") - -## Build test server for unit tests or benchmarks -#if(ARROW_BUILD_TESTS OR ARROW_BUILD_BENCHMARKS) -# add_executable(flight-test-server test_server.cc) -# target_link_libraries(flight-test-server ${ARROW_FLIGHT_TEST_LINK_LIBS} -# ${GFLAGS_LIBRARIES} GTest::gtest) -# -# if(ARROW_BUILD_TESTS) -# add_dependencies(arrow-flight-test flight-test-server) -# endif() -# -# add_dependencies(arrow_flight flight-test-server) -#endif() -# -#if(ARROW_BUILD_INTEGRATION) -# add_executable(flight-test-integration-server test_integration_server.cc) -# target_link_libraries(flight-test-integration-server ${ARROW_FLIGHT_TEST_LINK_LIBS} -# ${GFLAGS_LIBRARIES} GTest::gtest) -# -# add_executable(flight-test-integration-client test_integration_client.cc) -# target_link_libraries(flight-test-integration-client ${ARROW_FLIGHT_TEST_LINK_LIBS} -# ${GFLAGS_LIBRARIES} GTest::gtest) -# -# add_dependencies(arrow_flight flight-test-integration-client -# flight-test-integration-server) -# add_dependencies(arrow-integration flight-test-integration-client -# flight-test-integration-server) -#endif() -# -#if(ARROW_BUILD_BENCHMARKS) -# # Perf server for benchmarks -# set(PERF_PROTO_GENERATED_FILES "${CMAKE_CURRENT_BINARY_DIR}/perf.pb.cc" -# "${CMAKE_CURRENT_BINARY_DIR}/perf.pb.h") -# -# add_custom_command(OUTPUT ${PERF_PROTO_GENERATED_FILES} -# COMMAND ${ARROW_PROTOBUF_PROTOC} "-I${CMAKE_CURRENT_SOURCE_DIR}" -# "--cpp_out=${CMAKE_CURRENT_BINARY_DIR}" "perf.proto" -# DEPENDS ${PROTO_DEPENDS}) -# -# add_executable(arrow-flight-perf-server perf_server.cc perf.pb.cc) -# target_link_libraries(arrow-flight-perf-server ${ARROW_FLIGHT_TEST_LINK_LIBS} -# ${GFLAGS_LIBRARIES} GTest::gtest) -# -# add_executable(arrow-flight-benchmark flight_benchmark.cc perf.pb.cc) -# target_link_libraries(arrow-flight-benchmark ${ARROW_FLIGHT_TEST_LINK_LIBS} -# ${GFLAGS_LIBRARIES} GTest::gtest) -# -# add_dependencies(arrow-flight-benchmark arrow-flight-perf-server) -# -# add_dependencies(arrow_flight arrow-flight-benchmark) -#endif(ARROW_BUILD_BENCHMARKS) diff --git a/cpp/src/arrow/flight/flight-sql/arrow-flight-sql.pc.in b/cpp/src/arrow/flight/flight-sql/arrow-flight-sql.pc.in index 157aa766ede..6d4eab0b4a0 100644 --- a/cpp/src/arrow/flight/flight-sql/arrow-flight-sql.pc.in +++ b/cpp/src/arrow/flight/flight-sql/arrow-flight-sql.pc.in @@ -19,7 +19,7 @@ libdir=@CMAKE_INSTALL_FULL_LIBDIR@ includedir=@CMAKE_INSTALL_FULL_INCLUDEDIR@ Name: Apache Arrow Flight SQL -Description: 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/flight-sql/client.cc b/cpp/src/arrow/flight/flight-sql/client.cc index e4296c3e92e..7a6ea719628 100644 --- a/cpp/src/arrow/flight/flight-sql/client.cc +++ b/cpp/src/arrow/flight/flight-sql/client.cc @@ -18,7 +18,4 @@ #include "arrow/flight/flight-sql/client.h" #include "arrow/flight/flight-sql/FlightSql.pb.h" -void x() { - arrow::flight::protocol::sql::CommandStatementQuery command; - command.set_query(""); -} +// TODO: Implement diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index ff72c7bde99..c63a250f4b6 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -1,9 +1,22 @@ +// 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 // -// Created by rafael on 8/23/21. +// 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. -#ifndef ARROW_CLIENT_H -#define ARROW_CLIENT_H +#ifndef ARROW_FLIGHT_SQL_CLIENT_H +#define ARROW_FLIGHT_SQL_CLIENT_H class client { @@ -11,4 +24,4 @@ class client { }; -#endif //ARROW_CLIENT_H +#endif //ARROW_FLIGHT_SQL_CLIENT_H From f3fe962c5838c345589320d7c559526650e87cde Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 6 Sep 2021 11:37:08 -0300 Subject: [PATCH 004/237] Fix checkstyle issues --- cpp/cmake_modules/FindArrowFlightSql.cmake | 12 ++++++++---- cpp/src/arrow/flight/flight-sql/CMakeLists.txt | 3 +-- cpp/src/arrow/flight/flight-sql/client.h | 1 - cpp/src/arrow/flight/flight-sql/client_test.cc | 2 +- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/cpp/cmake_modules/FindArrowFlightSql.cmake b/cpp/cmake_modules/FindArrowFlightSql.cmake index 49471dd5a2e..a40af25c2a4 100644 --- a/cpp/cmake_modules/FindArrowFlightSql.cmake +++ b/cpp/cmake_modules/FindArrowFlightSql.cmake @@ -77,13 +77,17 @@ mark_as_advanced(ARROW_FLIGHT_SQL_IMPORT_LIB find_package_handle_standard_args( ArrowFlightSql - REQUIRED_VARS ARROW_FLIGHT_SQL_INCLUDE_DIR ARROW_FLIGHT_SQL_LIB_DIR ARROW_FLIGHT_SQL_VERSION_MATCH + 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}") + 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/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 407aa3dc056..75d4b6549c7 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -49,8 +49,7 @@ set(CMAKE_CXX_FLAGS_BACKUP "${CMAKE_CXX_FLAGS}") string(REPLACE "/WX" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") string(REPLACE "-Werror " " " CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") -set(ARROW_FLIGHT_SQL_SRCS - client.cc) +set(ARROW_FLIGHT_SQL_SRCS client.cc) add_arrow_lib(arrow_flight_sql CMAKE_PACKAGE_NAME diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index c63a250f4b6..b2015729116 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -20,7 +20,6 @@ class client { - }; diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 0dd029becb4..2e0ae092e43 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -23,4 +23,4 @@ TEST(TestFlightSql, Test) { ASSERT_EQ(1, 1); } -} \ No newline at end of file +} From fde5c0022594146b971a635c337616ffe86e5bd1 Mon Sep 17 00:00:00 2001 From: Jose Almeida <53087160+jcralmeida@users.noreply.github.com> Date: Tue, 14 Sep 2021 11:42:48 -0300 Subject: [PATCH 005/237] [C++] Implements methods from flight-sql-client (#120) * add a header file to the sql-client * Implements methods from sql-client * Add configuration files to the flight-sql * Change getFlightInfo to virtual and its constructor to protected * Create a mock test for getCatalogs * Remove unused test from CMakeLists.txt * Fix checkstyle on flight-sql files * Fix duplicate tests execution * Add test for getSchema from flightsql * Update flight headers and implements getTable and getTableTypes * Add other unit tests for metadata methods * Fix checkstyle errors * Implement missing methods GetPrimaryKeys, GetImportedKeys and GetExportedKeys * Refactor flight-sql/client.cc implementation * Remove unimplemented ExecuteUpdate test * Add google/protobuf/message include to flight-sql-client * Undo changes on flight/client.h and use templates for mocking FlightClient on FlightSqlClient * Use string references where parameters can not be null * Reorder FlightSqlClient method arguments * Avoid needing to use diamond syntax on FlightSqlClient Co-authored-by: Rafael Telles --- .../arrow/flight/flight-sql/CMakeLists.txt | 25 ++- cpp/src/arrow/flight/flight-sql/client.h | 145 ++++++++++++- cpp/src/arrow/flight/flight-sql/client_impl.h | 204 ++++++++++++++++++ .../arrow/flight/flight-sql/client_test.cc | 194 ++++++++++++++++- .../flight/flight-sql/protocol_internal.cc | 26 +++ .../{client.cc => protocol_internal.h} | 12 +- 6 files changed, 592 insertions(+), 14 deletions(-) create mode 100644 cpp/src/arrow/flight/flight-sql/client_impl.h create mode 100644 cpp/src/arrow/flight/flight-sql/protocol_internal.cc rename cpp/src/arrow/flight/flight-sql/{client.cc => protocol_internal.h} (68%) diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 75d4b6549c7..7960b6a6c0c 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -19,6 +19,8 @@ add_custom_target(arrow_flight_sql) arrow_install_all_headers("arrow/flight/flight-sql") +set(ARROW_FLIGHT_SQL_LINK_LIBS ${ARROW_PROTOBUF_LIBPROTOBUF}) + set(FLIGHT_SQL_PROTO_PATH "${ARROW_SOURCE_DIR}/../format") set(FLIGHT_SQL_PROTO ${ARROW_SOURCE_DIR}/../format/FlightSql.proto) @@ -49,7 +51,7 @@ set(CMAKE_CXX_FLAGS_BACKUP "${CMAKE_CXX_FLAGS}") string(REPLACE "/WX" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") string(REPLACE "-Werror " " " CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") -set(ARROW_FLIGHT_SQL_SRCS client.cc) +set(ARROW_FLIGHT_SQL_SRCS protocol_internal.cc) add_arrow_lib(arrow_flight_sql CMAKE_PACKAGE_NAME @@ -61,20 +63,31 @@ add_arrow_lib(arrow_flight_sql SOURCES ${ARROW_FLIGHT_SQL_SRCS} PRECOMPILED_HEADERS - "$<$:arrow/flight/pch.h>" + "$<$:arrow/flight/flight-sql/pch.h>" DEPENDENCIES flight_sql_grpc_gen SHARED_LINK_FLAGS ${ARROW_VERSION_SCRIPT_FLAGS} # Defined in cpp/arrow/CMakeLists.txt SHARED_LINK_LIBS - arrow_shared - ${ARROW_FLIGHT_LINK_LIBS} + arrow_flight_shared + ${ARROW_FLIGHT_SQL_LINK_LIBS} STATIC_LINK_LIBS - arrow_static - ${ARROW_FLIGHT_LINK_LIBS}) + arrow_flight_static + ${ARROW_FLIGHT_SQL_LINK_LIBS}) + +if(ARROW_TEST_LINKAGE STREQUAL "static") + set(ARROW_FLIGHT_SQL_TEST_LINK_LIBS + arrow_flight_sql_static ${ARROW_FLIGHT_STATIC_LINK_LIBS} ${ARROW_TEST_LINK_LIBS}) +else() + set(ARROW_FLIGHT_SQL_TEST_LINK_LIBS arrow_flight_sql_shared ${ARROW_TEST_LINK_LIBS}) +endif() add_arrow_test(flight_sql_test SOURCES client_test.cc + STATIC_LINK_LIBS + ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} + GTest::gtest + GTest::gmock LABELS "arrow_flight_sql") diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index b2015729116..c53fa8b8fc9 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -18,9 +18,150 @@ #ifndef ARROW_FLIGHT_SQL_CLIENT_H #define ARROW_FLIGHT_SQL_CLIENT_H +#include +#include +#include +#include -class client { +namespace arrow { +namespace flight { +namespace sql { +namespace internal { + +/// \brief Flight client with Flight SQL semantics. +template +class FlightSqlClientT { + public: + explicit FlightSqlClientT(T* client); + + ~FlightSqlClientT(); + + /// \brief Execute a query on the server. + /// \param[in] options RPC-layer hints for this call. + /// \param[out] flight_info The FlightInfo describing where to access the dataset + /// \param[in] query The query to be executed in the UTF-8 format. + /// \return Status. + Status Execute(const FlightCallOptions& options, + std::unique_ptr* flight_info, + const std::string& query); + + /// \brief Execute an update query on the server. + /// \param[in] options RPC-layer hints for this call. + /// \param[out] rows The quantity of rows affected by the operation. + /// \param[in] query The query to be executed in the UTF-8 format. + /// \return Status. + Status ExecuteUpdate(const FlightCallOptions& options, + int64_t* rows, + const std::string& query); + + /// \brief Request a list of catalogs. + /// \param[in] options RPC-layer hints for this call. + /// \param[out] flight_info The FlightInfo describing where to access the dataset. + /// \return Status. + Status GetCatalogs(const FlightCallOptions& options, + std::unique_ptr* flight_info); + + /// \brief Request a list of schemas. + /// \param[in] options RPC-layer hints for this call. + /// \param[out] flight_info The FlightInfo describing where to access the + /// dataset. + /// \param[in] catalog The catalog. + /// \param[in] schema_filter_pattern The schema filter pattern. + /// \return Status. + Status GetSchemas(const FlightCallOptions& options, + std::unique_ptr* flight_info, + std::string* catalog, + std::string* 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 + /// \param[out] stream the returned RecordBatchReader + /// \return Status + Status DoGet(const FlightCallOptions& options, + const Ticket& ticket, + std::unique_ptr* stream); + + /// \brief Request a list of tables. + /// \param[in] options RPC-layer hints for this call. + /// \param[out] flight_info The FlightInfo describing where to access the + /// dataset. + /// \param[in] catalog The catalog. + /// \param[in] 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 Status. + Status GetTables(const FlightCallOptions& options, + std::unique_ptr* flight_info, + const std::string* catalog, + const std::string* schema_filter_pattern, + const std::string* table_filter_pattern, + bool include_schema, + std::vector& table_types); + + /// \brief Request the primary keys for a table. + /// \param[in] options RPC-layer hints for this call. + /// \param[out] flight_info The FlightInfo describing where to access the dataset. + /// \param[in] catalog The catalog. + /// \param[in] schema The schema. + /// \param[in] table The table. + /// \return Status. + Status GetPrimaryKeys(const FlightCallOptions& options, + std::unique_ptr* flight_info, + const std::string* catalog, + const std::string* schema, + const std::string& table); + + /// \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[out] flight_info The FlightInfo describing where to access the dataset. + /// \param[in] catalog The foreign key table catalog. + /// \param[in] schema The foreign key table schema. + /// \param[in] table The foreign key table. Cannot be null. + /// \return Status. + Status GetExportedKeys(const FlightCallOptions& options, + std::unique_ptr* flight_info, + const std::string* catalog, + const std::string* schema, + const std::string& table); + + /// \brief Retrieves the foreign key columns for the given table. + /// \param[in] options RPC-layer hints for this call. + /// \param[out] flight_info The FlightInfo describing where to access the dataset. + /// \param[in] catalog The primary key table catalog. + /// \param[in] schema The primary key table schema. + /// \param[in] table The primary key table. Cannot be null. + /// \return Status. + Status GetImportedKeys(const FlightCallOptions& options, + std::unique_ptr* flight_info, + const std::string* catalog, + const std::string* schema, + const std::string& table); + + /// \brief Request a list of table types. + /// \param[in] options RPC-layer hints for this call. + /// \param[out] flight_info The FlightInfo describing where to access the dataset. + /// \return Status. + Status GetTableTypes(const FlightCallOptions& options, + std::unique_ptr* flight_info); + + + private: + T* client; }; +} // namespace internal + +typedef internal::FlightSqlClientT FlightSqlClient; + +} // namespace sql +} // namespace flight +} // namespace arrow + +#endif // ARROW_FLIGHT_SQL_CLIENT_H -#endif //ARROW_FLIGHT_SQL_CLIENT_H +#include diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h new file mode 100644 index 00000000000..86c79a7c4af --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -0,0 +1,204 @@ +// 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 + +namespace pb = arrow::flight::protocol; + +namespace arrow { +namespace flight { +namespace sql { +namespace internal { + +template +FlightSqlClientT::FlightSqlClientT(T *client) { + this->client = client; +} + +template +FlightSqlClientT::~FlightSqlClientT() = default; + +template +Status GetFlightInfoForCommand(T* client, + const FlightCallOptions &options, + std::unique_ptr *flight_info, + const google::protobuf::Message &command) { + google::protobuf::Any any; + any.PackFrom(command); + + const std::string& string = any.SerializeAsString(); + const FlightDescriptor& descriptor = FlightDescriptor::Command(string); + + return client->GetFlightInfo(options, descriptor, flight_info); +} + +template +Status FlightSqlClientT::Execute(const FlightCallOptions& options, + std::unique_ptr* flight_info, + const std::string& query) { + pb::sql::CommandStatementQuery command; + command.set_query(query); + + return GetFlightInfoForCommand(client, options, flight_info, command); +} + +template +Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, + int64_t* rows, + const std::string& query) { + return Status::NotImplemented("ExecuteUpdate"); +} + +template +Status FlightSqlClientT::GetCatalogs(const FlightCallOptions& options, + std::unique_ptr* flight_info) { + pb::sql::CommandGetCatalogs command; + + return GetFlightInfoForCommand(client, options, flight_info, command); +} + +template +Status FlightSqlClientT::GetSchemas(const FlightCallOptions& options, + std::unique_ptr* flight_info, + std::string* catalog, + std::string* schema_filter_pattern) { + pb::sql::CommandGetSchemas command; + if (catalog != NULLPTR) { + command.set_catalog(*catalog); + } + if (schema_filter_pattern != NULLPTR) { + command.set_schema_filter_pattern(*schema_filter_pattern); + } + + return GetFlightInfoForCommand(client, options, flight_info, command); +} + +template +Status FlightSqlClientT::GetTables(const FlightCallOptions& options, + std::unique_ptr* flight_info, + const std::string* catalog, + const std::string* schema_filter_pattern, + const std::string* table_filter_pattern, + bool include_schema, + std::vector& table_types) { + pb::sql::CommandGetTables command; + + if (catalog != NULLPTR) { + command.set_catalog(*catalog); + } + + if (schema_filter_pattern != NULLPTR) { + command.set_schema_filter_pattern(*schema_filter_pattern); + } + + if (table_filter_pattern != NULLPTR) { + command.set_table_name_filter_pattern(*table_filter_pattern); + } + + command.set_include_schema(include_schema); + + for (const std::string &table_type : table_types) { + command.add_table_types(table_type); + } + + return GetFlightInfoForCommand(client, options, flight_info, command); +} + +template +Status FlightSqlClientT::GetPrimaryKeys(const FlightCallOptions& options, + std::unique_ptr* flight_info, + const std::string* catalog, + const std::string* schema, + const std::string& table) { + pb::sql::CommandGetPrimaryKeys command; + + if (catalog != NULLPTR) { + command.set_catalog(*catalog); + } + + if (schema != NULLPTR) { + command.set_schema(*schema); + } + + command.set_table(table); + + return GetFlightInfoForCommand(client, options, flight_info, command); +} + +template +Status FlightSqlClientT::GetExportedKeys(const FlightCallOptions& options, + std::unique_ptr* flight_info, + const std::string* catalog, + const std::string* schema, + const std::string& table) { + pb::sql::CommandGetExportedKeys command; + + if (catalog != NULLPTR) { + command.set_catalog(*catalog); + } + + if (schema != NULLPTR) { + command.set_schema(*schema); + } + + command.set_table(table); + + return GetFlightInfoForCommand(client, options, flight_info, command); +} + +template +Status FlightSqlClientT::GetImportedKeys(const FlightCallOptions& options, + std::unique_ptr* flight_info, + const std::string* catalog, + const std::string* schema, + const std::string& table) { + pb::sql::CommandGetImportedKeys command; + + if (catalog != NULLPTR) { + command.set_catalog(*catalog); + } + + if (schema != NULLPTR) { + command.set_schema(*schema); + } + + command.set_table(table); + + return GetFlightInfoForCommand(client, options, flight_info, command); +} + +template +Status FlightSqlClientT::GetTableTypes(const FlightCallOptions& options, + std::unique_ptr* flight_info) { + pb::sql::CommandGetTableTypes command; + + return GetFlightInfoForCommand(client, options, flight_info, command); +} + +template +Status FlightSqlClientT::DoGet(const FlightCallOptions& options, + const Ticket& ticket, + std::unique_ptr* stream) { + return client->DoGet(options, ticket, stream); +} + +} // namespace internal +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 2e0ae092e43..9eedaa7fc7d 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -15,12 +15,202 @@ // specific language governing permissions and limitations // under the License. +#include +#include +#include +#include +#include +#include #include +namespace pb = arrow::flight::protocol; +using ::testing::Ref; + + namespace arrow { +namespace flight { +namespace sql { + +using internal::FlightSqlClientT; + +class FlightClientMock { + public: + MOCK_METHOD(Status, GetFlightInfo, (const FlightCallOptions&, const FlightDescriptor&, + std::unique_ptr*)); +}; + +FlightDescriptor getDescriptor(google::protobuf::Message &command) { + google::protobuf::Any any; + any.PackFrom(command); + + const std::string &string = any.SerializeAsString(); + return FlightDescriptor::Command(string); +} + +TEST(TestFlightSql, TestGetCatalogs) { + FlightClientMock flightClientMock; + FlightSqlClientT sqlClient(&flightClientMock); + FlightCallOptions call_options; + + pb::sql::CommandGetCatalogs command; + FlightDescriptor descriptor = getDescriptor(command); + + std::unique_ptr flight_info; + EXPECT_CALL(flightClientMock, + GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + + (void) sqlClient.GetCatalogs(call_options, &flight_info); +} + +TEST(TestFlightSql, TestGetSchemas) { + FlightClientMock flightClientMock; + FlightSqlClientT sqlClient(&flightClientMock); + FlightCallOptions call_options; + + std::string schema_filter_pattern = "schema_filter_pattern"; + std::string catalog = "catalog"; + + pb::sql::CommandGetSchemas command; + command.set_catalog(catalog); + command.set_schema_filter_pattern(schema_filter_pattern); + FlightDescriptor descriptor = getDescriptor(command); + + std::unique_ptr flight_info; + EXPECT_CALL(flightClientMock, + GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + + (void) sqlClient.GetSchemas(call_options, &flight_info, &catalog, + &schema_filter_pattern); +} + +TEST(TestFlightSql, TestGetTables) { + FlightClientMock flightClientMock; + FlightSqlClientT sqlClient(&flightClientMock); + FlightCallOptions call_options; + + 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_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); + + std::unique_ptr flight_info; + EXPECT_CALL(flightClientMock, + GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + + (void) sqlClient.GetTables(call_options, &flight_info, &catalog, &schema_filter_pattern, + &table_name_filter_pattern, include_schema, table_types); +} + +TEST(TestFlightSql, TestGetTableTypes) { + FlightClientMock flightClientMock; + FlightSqlClientT sqlClient(&flightClientMock); + FlightCallOptions call_options; -TEST(TestFlightSql, Test) { - ASSERT_EQ(1, 1); + pb::sql::CommandGetTableTypes command; + FlightDescriptor descriptor = getDescriptor(command); + + std::unique_ptr flight_info; + EXPECT_CALL(flightClientMock, + GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + + (void) sqlClient.GetTableTypes(call_options, &flight_info); } +TEST(TestFlightSql, TestGetExported) { + FlightClientMock flightClientMock; + FlightSqlClientT sqlClient(&flightClientMock); + FlightCallOptions call_options; + + std::string catalog = "catalog"; + std::string schema = "schema"; + std::string table = "table"; + + pb::sql::CommandGetExportedKeys command; + command.set_catalog(catalog); + command.set_schema(schema); + command.set_table(table); + FlightDescriptor descriptor = getDescriptor(command); + + std::unique_ptr flight_info; + EXPECT_CALL(flightClientMock, + GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + + (void) sqlClient.GetExportedKeys(call_options, &flight_info, &catalog, &schema, table); } + +TEST(TestFlightSql, TestGetImported) { + FlightClientMock flightClientMock; + FlightSqlClientT sqlClient(&flightClientMock); + FlightCallOptions call_options; + + std::string catalog = "catalog"; + std::string schema = "schema"; + std::string table = "table"; + + pb::sql::CommandGetImportedKeys command; + command.set_catalog(catalog); + command.set_schema(schema); + command.set_table(table); + FlightDescriptor descriptor = getDescriptor(command); + + std::unique_ptr flight_info; + EXPECT_CALL(flightClientMock, + GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + + (void) sqlClient.GetImportedKeys(call_options, &flight_info, &catalog, &schema, table); +} + +TEST(TestFlightSql, TestGetPrimary) { + FlightClientMock flightClientMock; + FlightSqlClientT sqlClient(&flightClientMock); + FlightCallOptions call_options; + + std::string catalog = "catalog"; + std::string schema = "schema"; + std::string table = "table"; + + pb::sql::CommandGetPrimaryKeys command; + command.set_catalog(catalog); + command.set_schema(schema); + command.set_table(table); + FlightDescriptor descriptor = getDescriptor(command); + + std::unique_ptr flight_info; + EXPECT_CALL(flightClientMock, + GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + + (void) sqlClient.GetPrimaryKeys(call_options, &flight_info, &catalog, &schema, table); +} + +TEST(TestFlightSql, TestExecute) { + FlightClientMock flightClientMock; + FlightSqlClientT sqlClient(&flightClientMock); + FlightCallOptions call_options; + + std::string query = "query"; + + pb::sql::CommandStatementQuery command; + command.set_query(query); + FlightDescriptor descriptor = getDescriptor(command); + + std::unique_ptr flight_info; + EXPECT_CALL(flightClientMock, + GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + + (void) sqlClient.Execute(call_options, &flight_info, query); +} + +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/protocol_internal.cc b/cpp/src/arrow/flight/flight-sql/protocol_internal.cc new file mode 100644 index 00000000000..851d6495863 --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/protocol_internal.cc @@ -0,0 +1,26 @@ +// 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 + +#include "arrow/flight/flight-sql/protocol_internal.h" + +// NOTE(wesm): Including .cc files in another .cc file would ordinarily be a +// no-no. We have customized the serialization path for FlightData, which is +// currently only possible through some pre-processor commands that need to be +// included before either of these files is compiled. Because we don't want to +// edit the generated C++ files, we include them here and do our gRPC +// customizations in protocol-internal.h +#include "arrow/flight/flight-sql/FlightSql.grpc.pb.cc" // NOLINT +#include "arrow/flight/flight-sql/FlightSql.pb.cc" // NOLINT diff --git a/cpp/src/arrow/flight/flight-sql/client.cc b/cpp/src/arrow/flight/flight-sql/protocol_internal.h similarity index 68% rename from cpp/src/arrow/flight/flight-sql/client.cc rename to cpp/src/arrow/flight/flight-sql/protocol_internal.h index 7a6ea719628..02ac1844285 100644 --- a/cpp/src/arrow/flight/flight-sql/client.cc +++ b/cpp/src/arrow/flight/flight-sql/protocol_internal.h @@ -13,9 +13,13 @@ // "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/flight-sql/client.h" -#include "arrow/flight/flight-sql/FlightSql.pb.h" +#pragma once -// TODO: Implement +// This header holds the Flight protobuf definitions. + +// Need to include this first to get our gRPC customizations +#include "arrow/flight/customize_protobuf.h" // IWYU pragma: export + +#include "arrow/flight/flight-sql/FlightSql.grpc.pb.h" // IWYU pragma: export +#include "arrow/flight/flight-sql/FlightSql.pb.h" // IWYU pragma: export From 69dbdecccf977d6b3a6fb33a76047ab368af546e Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 15 Sep 2021 19:33:25 -0300 Subject: [PATCH 006/237] [C++] Implement Flight SQL client test application (#121) * Implement Flight SQL client test application * Adjust PrintResults to consider multiple endpoints * Remove mentions to Dremio * Minor fix * Sort 'using' statements on test_app.cc * Transfer ownership of FLightClient to FlightSqlClient on constructor * Use reference instead of pointers on PrintResults * Make client methods to be const --- .../arrow/flight/flight-sql/CMakeLists.txt | 4 + cpp/src/arrow/flight/flight-sql/client.h | 24 +-- cpp/src/arrow/flight/flight-sql/client_impl.h | 26 +-- .../arrow/flight/flight-sql/client_test.cc | 56 +++--- cpp/src/arrow/flight/flight-sql/test_app.cc | 183 ++++++++++++++++++ 5 files changed, 244 insertions(+), 49 deletions(-) create mode 100644 cpp/src/arrow/flight/flight-sql/test_app.cc diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 7960b6a6c0c..5cc000171ac 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -91,3 +91,7 @@ add_arrow_test(flight_sql_test GTest::gmock LABELS "arrow_flight_sql") + +add_executable(flight_sql_test_app test_app.cc) +target_link_libraries(flight_sql_test_app PRIVATE arrow_flight_sql_static + ${GFLAGS_LIBRARIES}) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index c53fa8b8fc9..c26d866147f 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -32,7 +32,7 @@ namespace internal { template class FlightSqlClientT { public: - explicit FlightSqlClientT(T* client); + explicit FlightSqlClientT(std::unique_ptr &client); ~FlightSqlClientT(); @@ -43,7 +43,7 @@ class FlightSqlClientT { /// \return Status. Status Execute(const FlightCallOptions& options, std::unique_ptr* flight_info, - const std::string& query); + const std::string& query) const; /// \brief Execute an update query on the server. /// \param[in] options RPC-layer hints for this call. @@ -52,14 +52,14 @@ class FlightSqlClientT { /// \return Status. Status ExecuteUpdate(const FlightCallOptions& options, int64_t* rows, - const std::string& query); + const std::string& query) const; /// \brief Request a list of catalogs. /// \param[in] options RPC-layer hints for this call. /// \param[out] flight_info The FlightInfo describing where to access the dataset. /// \return Status. Status GetCatalogs(const FlightCallOptions& options, - std::unique_ptr* flight_info); + std::unique_ptr* flight_info) const; /// \brief Request a list of schemas. /// \param[in] options RPC-layer hints for this call. @@ -71,7 +71,7 @@ class FlightSqlClientT { Status GetSchemas(const FlightCallOptions& options, std::unique_ptr* flight_info, std::string* catalog, - std::string* schema_filter_pattern); + std::string* schema_filter_pattern) const; /// \brief Given a flight ticket and schema, request to be sent the /// stream. Returns record batch stream reader @@ -81,7 +81,7 @@ class FlightSqlClientT { /// \return Status Status DoGet(const FlightCallOptions& options, const Ticket& ticket, - std::unique_ptr* stream); + std::unique_ptr* stream) const; /// \brief Request a list of tables. /// \param[in] options RPC-layer hints for this call. @@ -100,7 +100,7 @@ class FlightSqlClientT { const std::string* schema_filter_pattern, const std::string* table_filter_pattern, bool include_schema, - std::vector& table_types); + std::vector& table_types) const; /// \brief Request the primary keys for a table. /// \param[in] options RPC-layer hints for this call. @@ -113,7 +113,7 @@ class FlightSqlClientT { std::unique_ptr* flight_info, const std::string* catalog, const std::string* schema, - const std::string& table); + const std::string& table) const; /// \brief Retrieves a description about the foreign key columns that reference the /// primary key columns of the given table. @@ -127,7 +127,7 @@ class FlightSqlClientT { std::unique_ptr* flight_info, const std::string* catalog, const std::string* schema, - const std::string& table); + const std::string& table) const; /// \brief Retrieves the foreign key columns for the given table. /// \param[in] options RPC-layer hints for this call. @@ -140,18 +140,18 @@ class FlightSqlClientT { std::unique_ptr* flight_info, const std::string* catalog, const std::string* schema, - const std::string& table); + const std::string& table) const; /// \brief Request a list of table types. /// \param[in] options RPC-layer hints for this call. /// \param[out] flight_info The FlightInfo describing where to access the dataset. /// \return Status. Status GetTableTypes(const FlightCallOptions& options, - std::unique_ptr* flight_info); + std::unique_ptr* flight_info) const; private: - T* client; + std::unique_ptr client; }; } // namespace internal diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 86c79a7c4af..1c0f5bb8dd9 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -27,15 +27,15 @@ namespace sql { namespace internal { template -FlightSqlClientT::FlightSqlClientT(T *client) { - this->client = client; +FlightSqlClientT::FlightSqlClientT(std::unique_ptr &client) { + this->client = std::move(client); } template FlightSqlClientT::~FlightSqlClientT() = default; template -Status GetFlightInfoForCommand(T* client, +Status GetFlightInfoForCommand(const std::unique_ptr &client, const FlightCallOptions &options, std::unique_ptr *flight_info, const google::protobuf::Message &command) { @@ -51,7 +51,7 @@ Status GetFlightInfoForCommand(T* client, template Status FlightSqlClientT::Execute(const FlightCallOptions& options, std::unique_ptr* flight_info, - const std::string& query) { + const std::string& query) const { pb::sql::CommandStatementQuery command; command.set_query(query); @@ -61,13 +61,13 @@ Status FlightSqlClientT::Execute(const FlightCallOptions& options, template Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, int64_t* rows, - const std::string& query) { + const std::string& query) const { return Status::NotImplemented("ExecuteUpdate"); } template Status FlightSqlClientT::GetCatalogs(const FlightCallOptions& options, - std::unique_ptr* flight_info) { + std::unique_ptr* flight_info) const { pb::sql::CommandGetCatalogs command; return GetFlightInfoForCommand(client, options, flight_info, command); @@ -77,7 +77,7 @@ template Status FlightSqlClientT::GetSchemas(const FlightCallOptions& options, std::unique_ptr* flight_info, std::string* catalog, - std::string* schema_filter_pattern) { + std::string* schema_filter_pattern) const { pb::sql::CommandGetSchemas command; if (catalog != NULLPTR) { command.set_catalog(*catalog); @@ -96,7 +96,7 @@ Status FlightSqlClientT::GetTables(const FlightCallOptions& options, const std::string* schema_filter_pattern, const std::string* table_filter_pattern, bool include_schema, - std::vector& table_types) { + std::vector& table_types) const { pb::sql::CommandGetTables command; if (catalog != NULLPTR) { @@ -125,7 +125,7 @@ Status FlightSqlClientT::GetPrimaryKeys(const FlightCallOptions& options, std::unique_ptr* flight_info, const std::string* catalog, const std::string* schema, - const std::string& table) { + const std::string& table) const { pb::sql::CommandGetPrimaryKeys command; if (catalog != NULLPTR) { @@ -146,7 +146,7 @@ Status FlightSqlClientT::GetExportedKeys(const FlightCallOptions& options, std::unique_ptr* flight_info, const std::string* catalog, const std::string* schema, - const std::string& table) { + const std::string& table) const { pb::sql::CommandGetExportedKeys command; if (catalog != NULLPTR) { @@ -167,7 +167,7 @@ Status FlightSqlClientT::GetImportedKeys(const FlightCallOptions& options, std::unique_ptr* flight_info, const std::string* catalog, const std::string* schema, - const std::string& table) { + const std::string& table) const { pb::sql::CommandGetImportedKeys command; if (catalog != NULLPTR) { @@ -185,7 +185,7 @@ Status FlightSqlClientT::GetImportedKeys(const FlightCallOptions& options, template Status FlightSqlClientT::GetTableTypes(const FlightCallOptions& options, - std::unique_ptr* flight_info) { + std::unique_ptr* flight_info) const { pb::sql::CommandGetTableTypes command; return GetFlightInfoForCommand(client, options, flight_info, command); @@ -194,7 +194,7 @@ Status FlightSqlClientT::GetTableTypes(const FlightCallOptions& options, template Status FlightSqlClientT::DoGet(const FlightCallOptions& options, const Ticket& ticket, - std::unique_ptr* stream) { + std::unique_ptr* stream) const { return client->DoGet(options, ticket, stream); } diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 9eedaa7fc7d..0ddeca9dbaf 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -48,23 +48,25 @@ FlightDescriptor getDescriptor(google::protobuf::Message &command) { } TEST(TestFlightSql, TestGetCatalogs) { - FlightClientMock flightClientMock; - FlightSqlClientT sqlClient(&flightClientMock); + auto *client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; pb::sql::CommandGetCatalogs command; FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(flightClientMock, + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); (void) sqlClient.GetCatalogs(call_options, &flight_info); } TEST(TestFlightSql, TestGetSchemas) { - FlightClientMock flightClientMock; - FlightSqlClientT sqlClient(&flightClientMock); + auto *client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; std::string schema_filter_pattern = "schema_filter_pattern"; @@ -76,7 +78,7 @@ TEST(TestFlightSql, TestGetSchemas) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(flightClientMock, + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); (void) sqlClient.GetSchemas(call_options, &flight_info, &catalog, @@ -84,8 +86,9 @@ TEST(TestFlightSql, TestGetSchemas) { } TEST(TestFlightSql, TestGetTables) { - FlightClientMock flightClientMock; - FlightSqlClientT sqlClient(&flightClientMock); + auto *client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; std::string catalog = "catalog"; @@ -105,7 +108,7 @@ TEST(TestFlightSql, TestGetTables) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(flightClientMock, + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); (void) sqlClient.GetTables(call_options, &flight_info, &catalog, &schema_filter_pattern, @@ -113,23 +116,25 @@ TEST(TestFlightSql, TestGetTables) { } TEST(TestFlightSql, TestGetTableTypes) { - FlightClientMock flightClientMock; - FlightSqlClientT sqlClient(&flightClientMock); + auto *client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; pb::sql::CommandGetTableTypes command; FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(flightClientMock, + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); (void) sqlClient.GetTableTypes(call_options, &flight_info); } TEST(TestFlightSql, TestGetExported) { - FlightClientMock flightClientMock; - FlightSqlClientT sqlClient(&flightClientMock); + auto *client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; std::string catalog = "catalog"; @@ -143,15 +148,16 @@ TEST(TestFlightSql, TestGetExported) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(flightClientMock, + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); (void) sqlClient.GetExportedKeys(call_options, &flight_info, &catalog, &schema, table); } TEST(TestFlightSql, TestGetImported) { - FlightClientMock flightClientMock; - FlightSqlClientT sqlClient(&flightClientMock); + auto *client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; std::string catalog = "catalog"; @@ -165,15 +171,16 @@ TEST(TestFlightSql, TestGetImported) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(flightClientMock, + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); (void) sqlClient.GetImportedKeys(call_options, &flight_info, &catalog, &schema, table); } TEST(TestFlightSql, TestGetPrimary) { - FlightClientMock flightClientMock; - FlightSqlClientT sqlClient(&flightClientMock); + auto *client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; std::string catalog = "catalog"; @@ -187,15 +194,16 @@ TEST(TestFlightSql, TestGetPrimary) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(flightClientMock, + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); (void) sqlClient.GetPrimaryKeys(call_options, &flight_info, &catalog, &schema, table); } TEST(TestFlightSql, TestExecute) { - FlightClientMock flightClientMock; - FlightSqlClientT sqlClient(&flightClientMock); + auto *client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; std::string query = "query"; @@ -205,7 +213,7 @@ TEST(TestFlightSql, TestExecute) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(flightClientMock, + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); (void) sqlClient.Execute(call_options, &flight_info, query); diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc new file mode 100644 index 00000000000..54e92f6a949 --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -0,0 +1,183 @@ +// 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 +#include +#include +#include +#include + +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; + +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(const FlightSqlClient &client, + const FlightCallOptions &call_options, + const FlightEndpoint &endpoint) { + std::unique_ptr stream; + ARROW_RETURN_NOT_OK(client.DoGet(call_options, endpoint.ticket, &stream)); + + 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(fLS::FLAGS_host, fLI::FLAGS_port, &location)); + ARROW_RETURN_NOT_OK(FlightClient::Connect(location, &client)); + + FlightCallOptions call_options; + + if (!fLS::FLAGS_username.empty() || !fLS::FLAGS_password.empty()) { + Result> bearer_result = + client->AuthenticateBasicToken({}, fLS::FLAGS_username, fLS::FLAGS_password); + ARROW_RETURN_NOT_OK(bearer_result); + + call_options.headers.push_back(bearer_result.ValueOrDie()); + } + + FlightSqlClient sqlClient(client); + + if (fLS::FLAGS_command == "ExecuteUpdate") { + int64_t rows = 0; + ARROW_RETURN_NOT_OK(sqlClient.ExecuteUpdate(call_options, &rows, fLS::FLAGS_query)); + + std::cout << "Result: " << rows << std::endl; + + return Status::OK(); + } + + std::unique_ptr info; + + if (fLS::FLAGS_command == "Execute") { + ARROW_RETURN_NOT_OK(sqlClient.Execute(call_options, &info, + fLS::FLAGS_query)); + } else if (fLS::FLAGS_command == "GetCatalogs") { + ARROW_RETURN_NOT_OK(sqlClient.GetCatalogs(call_options, &info)); + } else if (fLS::FLAGS_command == "GetSchemas") { + ARROW_RETURN_NOT_OK(sqlClient.GetSchemas(call_options, + &info, + &fLS::FLAGS_catalog, + &fLS::FLAGS_schema)); + } else if (fLS::FLAGS_command == "GetTableTypes") { + ARROW_RETURN_NOT_OK(sqlClient.GetTableTypes(call_options, &info)); + } else if (fLS::FLAGS_command == "GetTables") { + std::vector table_types = {}; + bool include_schema = false; + + ARROW_RETURN_NOT_OK(sqlClient.GetTables(call_options, &info, + &fLS::FLAGS_catalog, + &fLS::FLAGS_schema, + &fLS::FLAGS_table, + include_schema, + table_types)); + } else if (fLS::FLAGS_command == "GetExportedKeys") { + ARROW_RETURN_NOT_OK(sqlClient.GetExportedKeys(call_options, &info, + &fLS::FLAGS_catalog, + &fLS::FLAGS_schema, + fLS::FLAGS_table)); + } else if (fLS::FLAGS_command == "GetImportedKeys") { + ARROW_RETURN_NOT_OK(sqlClient.GetImportedKeys(call_options, &info, + &fLS::FLAGS_catalog, + &fLS::FLAGS_schema, + fLS::FLAGS_table)); + } else if (fLS::FLAGS_command == "GetPrimaryKeys") { + ARROW_RETURN_NOT_OK(sqlClient.GetPrimaryKeys(call_options, &info, + &fLS::FLAGS_catalog, + &fLS::FLAGS_schema, + fLS::FLAGS_table)); + } + + if (info != NULLPTR) { + return PrintResults(sqlClient, 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; +} From e078c057148c9ed91176036485e7c4a8363fb779 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 14 Sep 2021 16:55:50 -0300 Subject: [PATCH 007/237] Implement executeUpdate logic on sql client --- cpp/src/arrow/flight/flight-sql/client_impl.h | 31 ++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 1c0f5bb8dd9..9656f1652ac 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -18,6 +18,8 @@ #include #include #include +#include +#include namespace pb = arrow::flight::protocol; @@ -62,7 +64,34 @@ template Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, int64_t* rows, const std::string& query) const { - return Status::NotImplemented("ExecuteUpdate"); + pb::sql::CommandStatementUpdate command; + + command.set_query(query); + + google::protobuf::Any any; + any.PackFrom(command); + + const FlightDescriptor &descriptor = + FlightDescriptor::Command(any.SerializeAsString()); + + std::unique_ptr writer; + std::unique_ptr reader; + + const Status &put = client->DoPut(options, descriptor, NULL, &writer, &reader); + + std::shared_ptr metadata; + + const Status &status = reader->ReadMetadata(&metadata); + + pb::sql::DoPutUpdateResult doPutUpdateResult; + + Buffer *pBuffer = metadata.get(); + + const std::string &string = pBuffer->ToString(); + + doPutUpdateResult.ParseFrom(string); + *rows = doPutUpdateResult.record_count(); + return put; } template From 83db0122781b94c23557837661a27659ae8e4af7 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 14 Sep 2021 16:56:07 -0300 Subject: [PATCH 008/237] Create a mock test for execute_update method on sql_client --- cpp/src/arrow/flight/flight-sql/client_impl.h | 34 ++++----- .../arrow/flight/flight-sql/client_test.cc | 76 ++++++++++++++++++- 2 files changed, 89 insertions(+), 21 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 9656f1652ac..e50b0a6b008 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -64,34 +64,34 @@ template Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, int64_t* rows, const std::string& query) const { - pb::sql::CommandStatementUpdate command; + pb::sql::CommandStatementUpdate command; - command.set_query(query); + command.set_query(query); - google::protobuf::Any any; - any.PackFrom(command); + google::protobuf::Any any; + any.PackFrom(command); - const FlightDescriptor &descriptor = - FlightDescriptor::Command(any.SerializeAsString()); + const FlightDescriptor &descriptor = + FlightDescriptor::Command(any.SerializeAsString()); - std::unique_ptr writer; - std::unique_ptr reader; + std::unique_ptr writer; + std::unique_ptr reader; - const Status &put = client->DoPut(options, descriptor, NULL, &writer, &reader); + const Status &put = client->DoPut(options, descriptor, NULL, &writer, &reader); - std::shared_ptr metadata; + std::shared_ptr metadata; - const Status &status = reader->ReadMetadata(&metadata); + const Status &status = reader->ReadMetadata(&metadata); - pb::sql::DoPutUpdateResult doPutUpdateResult; + pb::sql::DoPutUpdateResult doPutUpdateResult; - Buffer *pBuffer = metadata.get(); + Buffer *pBuffer = metadata.get(); - const std::string &string = pBuffer->ToString(); + const std::string &string = pBuffer->ToString(); - doPutUpdateResult.ParseFrom(string); - *rows = doPutUpdateResult.record_count(); - return put; + doPutUpdateResult.ParseFrom(string); + *rows = doPutUpdateResult.record_count(); + return put; } template diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 0ddeca9dbaf..3b23b8e5ef8 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -24,6 +24,7 @@ #include namespace pb = arrow::flight::protocol; +using ::testing::_; using ::testing::Ref; @@ -35,15 +36,35 @@ using internal::FlightSqlClientT; class FlightClientMock { public: - MOCK_METHOD(Status, GetFlightInfo, (const FlightCallOptions&, const FlightDescriptor&, - std::unique_ptr*)); + MOCK_METHOD(Status, GetFlightInfo, + (const FlightCallOptions&, const FlightDescriptor&, + std::unique_ptr*)); + MOCK_METHOD(Status, DoPut, + (const FlightCallOptions&, const FlightDescriptor&, + const std::shared_ptr& schema, + std::unique_ptr*, + std::unique_ptr*)); }; -FlightDescriptor getDescriptor(google::protobuf::Message &command) { +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(); + } +}; + +FlightDescriptor getDescriptor(google::protobuf::Message& command) { google::protobuf::Any any; any.PackFrom(command); - const std::string &string = any.SerializeAsString(); + const std::string& string = any.SerializeAsString(); return FlightDescriptor::Command(string); } @@ -219,6 +240,53 @@ TEST(TestFlightSql, TestExecute) { (void) sqlClient.Execute(call_options, &flight_info, query); } +TEST(TestFlightSql, TestExecuteUpdate) { + auto *client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sqlClient(client_mock_ptr); + FlightCallOptions call_options; + + 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(*client_mock, 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(*client_mock, DoPut(Ref(call_options), descriptor, _, _, _)); + + int64_t num_rows = 0; + + (void) sqlClient.ExecuteUpdate(call_options, &num_rows, query); + + ASSERT_EQ(num_rows, 100); +} + } // namespace sql } // namespace flight } // namespace arrow From 015d015ea7da2f8347c63b76c55768656b8d8f27 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 16 Sep 2021 10:30:27 -0300 Subject: [PATCH 009/237] Refactor client_impl to reduce duplication on FlightDescriptor build --- cpp/src/arrow/flight/flight-sql/client_impl.h | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index e50b0a6b008..5f3ed1cb83b 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -36,16 +36,20 @@ FlightSqlClientT::FlightSqlClientT(std::unique_ptr &client) { template FlightSqlClientT::~FlightSqlClientT() = default; +FlightDescriptor GetFlightDescriptorForCommand(const google::protobuf::Message &command) { + google::protobuf::Any any; + any.PackFrom(command); + + const std::string& string = any.SerializeAsString(); + return FlightDescriptor::Command(string); +} + template Status GetFlightInfoForCommand(const std::unique_ptr &client, const FlightCallOptions &options, std::unique_ptr *flight_info, const google::protobuf::Message &command) { - google::protobuf::Any any; - any.PackFrom(command); - - const std::string& string = any.SerializeAsString(); - const FlightDescriptor& descriptor = FlightDescriptor::Command(string); + const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); return client->GetFlightInfo(options, descriptor, flight_info); } @@ -65,14 +69,9 @@ Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, int64_t* rows, const std::string& query) const { pb::sql::CommandStatementUpdate command; - command.set_query(query); - google::protobuf::Any any; - any.PackFrom(command); - - const FlightDescriptor &descriptor = - FlightDescriptor::Command(any.SerializeAsString()); + const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); std::unique_ptr writer; std::unique_ptr reader; From 7aa4c14425b5b2628e16488166dcd9fe762454be Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 17 Sep 2021 16:09:25 -0300 Subject: [PATCH 010/237] Change rows on executeUpdate to a unique_ptr --- cpp/src/arrow/flight/flight-sql/client_impl.h | 11 +++++++---- cpp/src/arrow/flight/flight-sql/client_test.cc | 6 +++--- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 5f3ed1cb83b..b556f307ffc 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -66,8 +66,8 @@ Status FlightSqlClientT::Execute(const FlightCallOptions& options, template Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, - int64_t* rows, - const std::string& query) const { + const std::string& query, + std::unique_ptr* rows) const { pb::sql::CommandStatementUpdate command; command.set_query(query); @@ -89,8 +89,11 @@ Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, const std::string &string = pBuffer->ToString(); doPutUpdateResult.ParseFrom(string); - *rows = doPutUpdateResult.record_count(); - return put; + rows->reset(new int64_t); + + **rows = doPutUpdateResult.record_count(); + + return Status::OK(); } template diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 3b23b8e5ef8..2b256e8214d 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -280,11 +280,11 @@ TEST(TestFlightSql, TestExecuteUpdate) { std::unique_ptr reader; EXPECT_CALL(*client_mock, DoPut(Ref(call_options), descriptor, _, _, _)); - int64_t num_rows = 0; + std::unique_ptr num_rows; - (void) sqlClient.ExecuteUpdate(call_options, &num_rows, query); + (void) sqlClient.ExecuteUpdate(call_options, query, &num_rows); - ASSERT_EQ(num_rows, 100); + ASSERT_EQ(*num_rows, 100); } } // namespace sql From 59e1c2897fc2adb0891072faad09c92745bb18c5 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 17 Sep 2021 16:11:36 -0300 Subject: [PATCH 011/237] Change order of output parameters on methods from flight-sql --- cpp/src/arrow/flight/flight-sql/client.h | 72 ++++++++----------- cpp/src/arrow/flight/flight-sql/client_impl.h | 72 +++++++++---------- .../arrow/flight/flight-sql/client_test.cc | 17 ++--- cpp/src/arrow/flight/flight-sql/test_app.cc | 24 +++---- 4 files changed, 85 insertions(+), 100 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index c26d866147f..2f0886bc9fe 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -38,21 +38,19 @@ class FlightSqlClientT { /// \brief Execute a query on the server. /// \param[in] options RPC-layer hints for this call. - /// \param[out] flight_info The FlightInfo describing where to access the dataset /// \param[in] query The query to be executed in the UTF-8 format. + /// \param[out] flight_info The FlightInfo describing where to access the dataset /// \return Status. - Status Execute(const FlightCallOptions& options, - std::unique_ptr* flight_info, - const std::string& query) const; + Status Execute(const FlightCallOptions& options, const std::string& query, + std::unique_ptr* flight_info) const; /// \brief Execute an update query on the server. /// \param[in] options RPC-layer hints for this call. - /// \param[out] rows The quantity of rows affected by the operation. /// \param[in] query The query to be executed in the UTF-8 format. + /// \param[out] rows The quantity of rows affected by the operation. /// \return Status. - Status ExecuteUpdate(const FlightCallOptions& options, - int64_t* rows, - const std::string& query) const; + Status ExecuteUpdate(const FlightCallOptions& options, const std::string& query, + std::unique_ptr* rows) const; /// \brief Request a list of catalogs. /// \param[in] options RPC-layer hints for this call. @@ -63,15 +61,14 @@ class FlightSqlClientT { /// \brief Request a list of schemas. /// \param[in] options RPC-layer hints for this call. - /// \param[out] flight_info The FlightInfo describing where to access the - /// dataset. /// \param[in] catalog The catalog. /// \param[in] schema_filter_pattern The schema filter pattern. + /// \param[out] flight_info The FlightInfo describing where to access the + /// dataset. /// \return Status. - Status GetSchemas(const FlightCallOptions& options, - std::unique_ptr* flight_info, - std::string* catalog, - std::string* schema_filter_pattern) const; + Status GetSchemas(const FlightCallOptions& options, std::string* catalog, + std::string* schema_filter_pattern, + std::unique_ptr* flight_info) const; /// \brief Given a flight ticket and schema, request to be sent the /// stream. Returns record batch stream reader @@ -79,68 +76,59 @@ class FlightSqlClientT { /// \param[in] ticket The flight ticket to use /// \param[out] stream the returned RecordBatchReader /// \return Status - Status DoGet(const FlightCallOptions& options, - const Ticket& ticket, + Status DoGet(const FlightCallOptions& options, const Ticket& ticket, std::unique_ptr* stream) const; /// \brief Request a list of tables. /// \param[in] options RPC-layer hints for this call. - /// \param[out] flight_info The FlightInfo describing where to access the - /// dataset. /// \param[in] catalog The catalog. /// \param[in] 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. + /// \param[out] flight_info The FlightInfo describing where to access the + /// dataset. /// \return Status. - Status GetTables(const FlightCallOptions& options, - std::unique_ptr* flight_info, - const std::string* catalog, + Status GetTables(const FlightCallOptions& options, const std::string* catalog, const std::string* schema_filter_pattern, - const std::string* table_filter_pattern, - bool include_schema, - std::vector& table_types) const; + const std::string* table_filter_pattern, bool include_schema, + std::vector& table_types, + std::unique_ptr* flight_info) const; /// \brief Request the primary keys for a table. /// \param[in] options RPC-layer hints for this call. - /// \param[out] flight_info The FlightInfo describing where to access the dataset. /// \param[in] catalog The catalog. /// \param[in] schema The schema. /// \param[in] table The table. + /// \param[out] flight_info The FlightInfo describing where to access the dataset. /// \return Status. - Status GetPrimaryKeys(const FlightCallOptions& options, - std::unique_ptr* flight_info, - const std::string* catalog, - const std::string* schema, - const std::string& table) const; + Status GetPrimaryKeys(const FlightCallOptions& options, const std::string* catalog, + const std::string* schema, const std::string& table, + std::unique_ptr* flight_info) const; /// \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[out] flight_info The FlightInfo describing where to access the dataset. /// \param[in] catalog The foreign key table catalog. /// \param[in] schema The foreign key table schema. /// \param[in] table The foreign key table. Cannot be null. + /// \param[out] flight_info The FlightInfo describing where to access the dataset. /// \return Status. - Status GetExportedKeys(const FlightCallOptions& options, - std::unique_ptr* flight_info, - const std::string* catalog, - const std::string* schema, - const std::string& table) const; + Status GetExportedKeys(const FlightCallOptions& options, const std::string* catalog, + const std::string* schema, const std::string& table, + std::unique_ptr* flight_info) const; /// \brief Retrieves the foreign key columns for the given table. /// \param[in] options RPC-layer hints for this call. - /// \param[out] flight_info The FlightInfo describing where to access the dataset. /// \param[in] catalog The primary key table catalog. /// \param[in] schema The primary key table schema. /// \param[in] table The primary key table. Cannot be null. + /// \param[out] flight_info The FlightInfo describing where to access the dataset. /// \return Status. - Status GetImportedKeys(const FlightCallOptions& options, - std::unique_ptr* flight_info, - const std::string* catalog, - const std::string* schema, - const std::string& table) const; + Status GetImportedKeys(const FlightCallOptions& options, const std::string* catalog, + const std::string* schema, const std::string& table, + std::unique_ptr* flight_info) const; /// \brief Request a list of table types. /// \param[in] options RPC-layer hints for this call. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index b556f307ffc..776f7fcf125 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -56,15 +56,15 @@ Status GetFlightInfoForCommand(const std::unique_ptr &client, template Status FlightSqlClientT::Execute(const FlightCallOptions& options, - std::unique_ptr* flight_info, - const std::string& query) const { + const std::string& query, + std::unique_ptr* flight_info) const { pb::sql::CommandStatementQuery command; command.set_query(query); return GetFlightInfoForCommand(client, options, flight_info, command); } -template +template Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, const std::string& query, std::unique_ptr* rows) const { @@ -98,7 +98,7 @@ Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, template Status FlightSqlClientT::GetCatalogs(const FlightCallOptions& options, - std::unique_ptr* flight_info) const { + std::unique_ptr* flight_info) const { pb::sql::CommandGetCatalogs command; return GetFlightInfoForCommand(client, options, flight_info, command); @@ -106,9 +106,9 @@ Status FlightSqlClientT::GetCatalogs(const FlightCallOptions& options, template Status FlightSqlClientT::GetSchemas(const FlightCallOptions& options, - std::unique_ptr* flight_info, - std::string* catalog, - std::string* schema_filter_pattern) const { + std::string* catalog, + std::string* schema_filter_pattern, + std::unique_ptr* flight_info) const { pb::sql::CommandGetSchemas command; if (catalog != NULLPTR) { command.set_catalog(*catalog); @@ -122,12 +122,12 @@ Status FlightSqlClientT::GetSchemas(const FlightCallOptions& options, template Status FlightSqlClientT::GetTables(const FlightCallOptions& options, - std::unique_ptr* flight_info, - const std::string* catalog, - const std::string* schema_filter_pattern, - const std::string* table_filter_pattern, - bool include_schema, - std::vector& table_types) const { + const std::string* catalog, + const std::string* schema_filter_pattern, + const std::string* table_filter_pattern, + bool include_schema, + std::vector& table_types, + std::unique_ptr* flight_info) const { pb::sql::CommandGetTables command; if (catalog != NULLPTR) { @@ -151,12 +151,11 @@ Status FlightSqlClientT::GetTables(const FlightCallOptions& options, return GetFlightInfoForCommand(client, options, flight_info, command); } -template -Status FlightSqlClientT::GetPrimaryKeys(const FlightCallOptions& options, - std::unique_ptr* flight_info, - const std::string* catalog, - const std::string* schema, - const std::string& table) const { +template +Status FlightSqlClientT::GetPrimaryKeys( + const FlightCallOptions& options, const std::string* catalog, + const std::string* schema, const std::string& table, + std::unique_ptr* flight_info) const { pb::sql::CommandGetPrimaryKeys command; if (catalog != NULLPTR) { @@ -172,12 +171,11 @@ Status FlightSqlClientT::GetPrimaryKeys(const FlightCallOptions& options, return GetFlightInfoForCommand(client, options, flight_info, command); } -template -Status FlightSqlClientT::GetExportedKeys(const FlightCallOptions& options, - std::unique_ptr* flight_info, - const std::string* catalog, - const std::string* schema, - const std::string& table) const { +template +Status FlightSqlClientT::GetExportedKeys( + const FlightCallOptions& options, const std::string* catalog, + const std::string* schema, const std::string& table, + std::unique_ptr* flight_info) const { pb::sql::CommandGetExportedKeys command; if (catalog != NULLPTR) { @@ -193,12 +191,11 @@ Status FlightSqlClientT::GetExportedKeys(const FlightCallOptions& options, return GetFlightInfoForCommand(client, options, flight_info, command); } -template -Status FlightSqlClientT::GetImportedKeys(const FlightCallOptions& options, - std::unique_ptr* flight_info, - const std::string* catalog, - const std::string* schema, - const std::string& table) const { +template +Status FlightSqlClientT::GetImportedKeys( + const FlightCallOptions& options, const std::string* catalog, + const std::string* schema, const std::string& table, + std::unique_ptr* flight_info) const { pb::sql::CommandGetImportedKeys command; if (catalog != NULLPTR) { @@ -214,18 +211,17 @@ Status FlightSqlClientT::GetImportedKeys(const FlightCallOptions& options, return GetFlightInfoForCommand(client, options, flight_info, command); } -template -Status FlightSqlClientT::GetTableTypes(const FlightCallOptions& options, - std::unique_ptr* flight_info) const { +template +Status FlightSqlClientT::GetTableTypes( + const FlightCallOptions& options, std::unique_ptr* flight_info) const { pb::sql::CommandGetTableTypes command; return GetFlightInfoForCommand(client, options, flight_info, command); } -template -Status FlightSqlClientT::DoGet(const FlightCallOptions& options, - const Ticket& ticket, - std::unique_ptr* stream) const { +template +Status FlightSqlClientT::DoGet(const FlightCallOptions& options, const Ticket& ticket, + std::unique_ptr* stream) const { return client->DoGet(options, ticket, stream); } diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 2b256e8214d..072c64b9755 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -102,8 +102,8 @@ TEST(TestFlightSql, TestGetSchemas) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.GetSchemas(call_options, &flight_info, &catalog, - &schema_filter_pattern); + (void) sqlClient.GetSchemas(call_options, &catalog, + &schema_filter_pattern, &flight_info); } TEST(TestFlightSql, TestGetTables) { @@ -132,8 +132,9 @@ TEST(TestFlightSql, TestGetTables) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.GetTables(call_options, &flight_info, &catalog, &schema_filter_pattern, - &table_name_filter_pattern, include_schema, table_types); + (void) sqlClient.GetTables(call_options, &catalog, &schema_filter_pattern, + &table_name_filter_pattern, include_schema, + table_types, &flight_info); } TEST(TestFlightSql, TestGetTableTypes) { @@ -172,7 +173,7 @@ TEST(TestFlightSql, TestGetExported) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.GetExportedKeys(call_options, &flight_info, &catalog, &schema, table); + (void) sqlClient.GetExportedKeys(call_options, &catalog, &schema, table, &flight_info); } TEST(TestFlightSql, TestGetImported) { @@ -195,7 +196,7 @@ TEST(TestFlightSql, TestGetImported) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.GetImportedKeys(call_options, &flight_info, &catalog, &schema, table); + (void) sqlClient.GetImportedKeys(call_options, &catalog, &schema, table, &flight_info); } TEST(TestFlightSql, TestGetPrimary) { @@ -218,7 +219,7 @@ TEST(TestFlightSql, TestGetPrimary) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.GetPrimaryKeys(call_options, &flight_info, &catalog, &schema, table); + (void) sqlClient.GetPrimaryKeys(call_options, &catalog, &schema, table, &flight_info); } TEST(TestFlightSql, TestExecute) { @@ -237,7 +238,7 @@ TEST(TestFlightSql, TestExecute) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.Execute(call_options, &flight_info, query); + (void) sqlClient.Execute(call_options, query, &flight_info); } TEST(TestFlightSql, TestExecuteUpdate) { diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index 54e92f6a949..f421af22d4a 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -126,42 +126,42 @@ Status RunMain() { std::unique_ptr info; if (fLS::FLAGS_command == "Execute") { - ARROW_RETURN_NOT_OK(sqlClient.Execute(call_options, &info, - fLS::FLAGS_query)); + ARROW_RETURN_NOT_OK(sqlClient.Execute(call_options, + fLS::FLAGS_query, &info)); } else if (fLS::FLAGS_command == "GetCatalogs") { ARROW_RETURN_NOT_OK(sqlClient.GetCatalogs(call_options, &info)); } else if (fLS::FLAGS_command == "GetSchemas") { ARROW_RETURN_NOT_OK(sqlClient.GetSchemas(call_options, - &info, &fLS::FLAGS_catalog, - &fLS::FLAGS_schema)); + &fLS::FLAGS_schema, + &info)); } else if (fLS::FLAGS_command == "GetTableTypes") { ARROW_RETURN_NOT_OK(sqlClient.GetTableTypes(call_options, &info)); } else if (fLS::FLAGS_command == "GetTables") { std::vector table_types = {}; bool include_schema = false; - ARROW_RETURN_NOT_OK(sqlClient.GetTables(call_options, &info, + ARROW_RETURN_NOT_OK(sqlClient.GetTables(call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, &fLS::FLAGS_table, include_schema, - table_types)); + table_types, &info)); } else if (fLS::FLAGS_command == "GetExportedKeys") { - ARROW_RETURN_NOT_OK(sqlClient.GetExportedKeys(call_options, &info, + ARROW_RETURN_NOT_OK(sqlClient.GetExportedKeys(call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, - fLS::FLAGS_table)); + fLS::FLAGS_table, &info)); } else if (fLS::FLAGS_command == "GetImportedKeys") { - ARROW_RETURN_NOT_OK(sqlClient.GetImportedKeys(call_options, &info, + ARROW_RETURN_NOT_OK(sqlClient.GetImportedKeys(call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, - fLS::FLAGS_table)); + fLS::FLAGS_table, &info)); } else if (fLS::FLAGS_command == "GetPrimaryKeys") { - ARROW_RETURN_NOT_OK(sqlClient.GetPrimaryKeys(call_options, &info, + ARROW_RETURN_NOT_OK(sqlClient.GetPrimaryKeys(call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, - fLS::FLAGS_table)); + fLS::FLAGS_table, &info)); } if (info != NULLPTR) { From 5e11c8243312d6c1dc7cb4723ada6f2cff739940 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 17 Sep 2021 16:11:47 -0300 Subject: [PATCH 012/237] Checkstyle fix on flight-sql --- cpp/src/arrow/flight/flight-sql/client.h | 3 +- cpp/src/arrow/flight/flight-sql/client_impl.h | 39 +++++++++---------- 2 files changed, 20 insertions(+), 22 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 2f0886bc9fe..cc6b94e368a 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -32,7 +32,7 @@ namespace internal { template class FlightSqlClientT { public: - explicit FlightSqlClientT(std::unique_ptr &client); + explicit FlightSqlClientT(std::unique_ptr& client); ~FlightSqlClientT(); @@ -137,7 +137,6 @@ class FlightSqlClientT { Status GetTableTypes(const FlightCallOptions& options, std::unique_ptr* flight_info) const; - private: std::unique_ptr client; }; diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 776f7fcf125..a18fb1d5033 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. +#include #include #include #include #include -#include namespace pb = arrow::flight::protocol; @@ -28,15 +28,15 @@ namespace flight { namespace sql { namespace internal { -template -FlightSqlClientT::FlightSqlClientT(std::unique_ptr &client) { +template +FlightSqlClientT::FlightSqlClientT(std::unique_ptr& client) { this->client = std::move(client); } -template +template FlightSqlClientT::~FlightSqlClientT() = default; -FlightDescriptor GetFlightDescriptorForCommand(const google::protobuf::Message &command) { +FlightDescriptor GetFlightDescriptorForCommand(const google::protobuf::Message& command) { google::protobuf::Any any; any.PackFrom(command); @@ -44,17 +44,17 @@ FlightDescriptor GetFlightDescriptorForCommand(const google::protobuf::Message & return FlightDescriptor::Command(string); } -template -Status GetFlightInfoForCommand(const std::unique_ptr &client, - const FlightCallOptions &options, - std::unique_ptr *flight_info, - const google::protobuf::Message &command) { +template +Status GetFlightInfoForCommand(const std::unique_ptr& client, + const FlightCallOptions& options, + std::unique_ptr* flight_info, + const google::protobuf::Message& command) { const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); return client->GetFlightInfo(options, descriptor, flight_info); } -template +template Status FlightSqlClientT::Execute(const FlightCallOptions& options, const std::string& query, std::unique_ptr* flight_info) const { @@ -73,20 +73,19 @@ Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); - std::unique_ptr writer; std::unique_ptr reader; - const Status &put = client->DoPut(options, descriptor, NULL, &writer, &reader); + ARROW_RETURN_NOT_OK(client->DoPut(options, descriptor, NULLPTR, NULL, &reader)); std::shared_ptr metadata; - const Status &status = reader->ReadMetadata(&metadata); + const Status& status = reader->ReadMetadata(&metadata); pb::sql::DoPutUpdateResult doPutUpdateResult; - Buffer *pBuffer = metadata.get(); + Buffer* pBuffer = metadata.get(); - const std::string &string = pBuffer->ToString(); + const std::string& string = pBuffer->ToString(); doPutUpdateResult.ParseFrom(string); rows->reset(new int64_t); @@ -96,7 +95,7 @@ Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, return Status::OK(); } -template +template Status FlightSqlClientT::GetCatalogs(const FlightCallOptions& options, std::unique_ptr* flight_info) const { pb::sql::CommandGetCatalogs command; @@ -104,7 +103,7 @@ Status FlightSqlClientT::GetCatalogs(const FlightCallOptions& options, return GetFlightInfoForCommand(client, options, flight_info, command); } -template +template Status FlightSqlClientT::GetSchemas(const FlightCallOptions& options, std::string* catalog, std::string* schema_filter_pattern, @@ -120,7 +119,7 @@ Status FlightSqlClientT::GetSchemas(const FlightCallOptions& options, return GetFlightInfoForCommand(client, options, flight_info, command); } -template +template Status FlightSqlClientT::GetTables(const FlightCallOptions& options, const std::string* catalog, const std::string* schema_filter_pattern, @@ -144,7 +143,7 @@ Status FlightSqlClientT::GetTables(const FlightCallOptions& options, command.set_include_schema(include_schema); - for (const std::string &table_type : table_types) { + for (const std::string& table_type : table_types) { command.add_table_types(table_type); } From bd2890ee117d64bca9856b3705eadd4eecd95274 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 17 Sep 2021 16:24:24 -0300 Subject: [PATCH 013/237] Change order of call from ExecuteUpdate on test_app.cc --- cpp/src/arrow/flight/flight-sql/test_app.cc | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index f421af22d4a..69c4ac25baf 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -115,10 +115,10 @@ Status RunMain() { FlightSqlClient sqlClient(client); if (fLS::FLAGS_command == "ExecuteUpdate") { - int64_t rows = 0; - ARROW_RETURN_NOT_OK(sqlClient.ExecuteUpdate(call_options, &rows, fLS::FLAGS_query)); + std::unique_ptr rows; + ARROW_RETURN_NOT_OK(sqlClient.ExecuteUpdate(call_options, fLS::FLAGS_query, &rows)); - std::cout << "Result: " << rows << std::endl; + std::cout << "Result: " << *rows << std::endl; return Status::OK(); } From bf5cfeff437e045cbf38945c82f64bcb8fbc5f2f Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 20 Sep 2021 15:25:42 -0300 Subject: [PATCH 014/237] Change ExecuteUpdate output argument to raw pointer instead of unique_ptr --- cpp/src/arrow/flight/flight-sql/client.h | 2 +- cpp/src/arrow/flight/flight-sql/client_impl.h | 6 ++---- cpp/src/arrow/flight/flight-sql/client_test.cc | 4 ++-- cpp/src/arrow/flight/flight-sql/test_app.cc | 4 ++-- 4 files changed, 7 insertions(+), 9 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index cc6b94e368a..b2b17318217 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -50,7 +50,7 @@ class FlightSqlClientT { /// \param[out] rows The quantity of rows affected by the operation. /// \return Status. Status ExecuteUpdate(const FlightCallOptions& options, const std::string& query, - std::unique_ptr* rows) const; + int64_t* rows) const; /// \brief Request a list of catalogs. /// \param[in] options RPC-layer hints for this call. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index a18fb1d5033..3abf9ba02b6 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -67,7 +67,7 @@ Status FlightSqlClientT::Execute(const FlightCallOptions& options, template Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, const std::string& query, - std::unique_ptr* rows) const { + int64_t* rows) const { pb::sql::CommandStatementUpdate command; command.set_query(query); @@ -88,9 +88,7 @@ Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, const std::string& string = pBuffer->ToString(); doPutUpdateResult.ParseFrom(string); - rows->reset(new int64_t); - - **rows = doPutUpdateResult.record_count(); + *rows = doPutUpdateResult.record_count(); return Status::OK(); } diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 072c64b9755..66a3251c2da 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -281,11 +281,11 @@ TEST(TestFlightSql, TestExecuteUpdate) { std::unique_ptr reader; EXPECT_CALL(*client_mock, DoPut(Ref(call_options), descriptor, _, _, _)); - std::unique_ptr num_rows; + int64_t num_rows; (void) sqlClient.ExecuteUpdate(call_options, query, &num_rows); - ASSERT_EQ(*num_rows, 100); + ASSERT_EQ(num_rows, 100); } } // namespace sql diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index 69c4ac25baf..9d602cbc69a 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -115,10 +115,10 @@ Status RunMain() { FlightSqlClient sqlClient(client); if (fLS::FLAGS_command == "ExecuteUpdate") { - std::unique_ptr rows; + int64_t rows; ARROW_RETURN_NOT_OK(sqlClient.ExecuteUpdate(call_options, fLS::FLAGS_query, &rows)); - std::cout << "Result: " << *rows << std::endl; + std::cout << "Result: " << rows << std::endl; return Status::OK(); } From 03425db52435102f9fa40e20da3d497c4d0410f8 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 22 Sep 2021 13:52:52 -0300 Subject: [PATCH 015/237] Add flight-sql server header file --- .../arrow/flight/flight-sql/CMakeLists.txt | 2 +- cpp/src/arrow/flight/flight-sql/sql_server.h | 328 ++++++++++++++++++ 2 files changed, 329 insertions(+), 1 deletion(-) create mode 100644 cpp/src/arrow/flight/flight-sql/sql_server.h diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 5cc000171ac..3fe10183950 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -92,6 +92,6 @@ add_arrow_test(flight_sql_test LABELS "arrow_flight_sql") -add_executable(flight_sql_test_app test_app.cc) +add_executable(flight_sql_test_app test_app.cc sql_server.h) target_link_libraries(flight_sql_test_app PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES}) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h new file mode 100644 index 00000000000..b2e27b16efc --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -0,0 +1,328 @@ +// 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 "../server.h" + +namespace pb = arrow::flight::protocol; + +namespace arrow { +namespace flight { +namespace sql { + +class FlightSqlServerBase : public FlightServerBase { + public: + FlightSqlServerBase(){}; + + ~FlightSqlServerBase() = default; + + Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, + std::unique_ptr* info) override { + google::protobuf::Any any; + any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size())); + + if (any.Is()) { + pb::sql::CommandStatementQuery command; + any.UnpackTo(&command); + + GetFlightInfoStatement(command, context, request, info); + } else if (any.Is()) { + std::cout << "Update" << std::endl; + } + + return Status::OK(); + }; + + Status DoGet(const ServerCallContext& context, const Ticket& request, + std::unique_ptr* stream) override { + google::protobuf::Any anyCommand; + + anyCommand.ParseFromArray(request.ticket.data(), + static_cast(request.ticket.size())); + + if (anyCommand.Is()) { + pb::sql::TicketStatementQuery command; + anyCommand.UnpackTo(&command); + + DoGetStatement(command, context, request, stream); + } + + return Status::OK(); + } + + /// \brief Gets information about a particular SQL query based data stream. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out] info The FlightInfo describing where to access the + /// dataset. + /// \return Status. + virtual Status GetFlightInfoStatement(const pb::sql::CommandStatementQuery& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info){}; + + /// \brief Returns data for a SQL query based data stream. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out]result The Result iterator. + /// \return Status. + virtual Status DoGetStatement(const pb::sql::TicketStatementQuery& command, + const ServerCallContext& context, const Ticket& ticket, + std::unique_ptr* result){}; + + /// \brief Gets information about a particular prepared statement data stream. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out] info The FlightInfo describing where to access the + /// dataset. + /// \return Status. + virtual Status GetFlightInfoPreparedStatement( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, const FlightDescriptor& descriptor, + std::unique_ptr* info){}; + + virtual Status DoGetPreparedStatement( + const pb::sql::CommandPreparedStatementQuery& command, const Ticket& ticket, + const ServerCallContext& context, std::unique_ptr* result){}; + + // TODO check doPut + virtual Status DoPutStatement(const pb::sql::CommandStatementUpdate& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor){}; + + // TODO check doPut + virtual Status DoPutPreparedStatementUpdate( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, const FlightDescriptor& descriptor){}; + + // TODO check doPut + virtual Status DoPutPreparedStatementQuery( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, const FlightDescriptor& descriptor){}; + + /// \brief Returns a FlightInfo object which contains the data to access the + /// stream of data. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out] info The FlightInfo describing where to access the + /// dataset. + /// \return Status. + virtual Status GetFlightInfoCatalogs(const pb::sql::CommandGetCatalogs& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info){}; + + /// \brief Returns data for catalogs based on a data stream. + /// \param[in] ticket The application-defined ticket identifying this stream. + /// \param[in] context Per-call context. + /// \param[out] result An interface for sending data back to the client. + /// \return Status. + virtual Status DoGetCatalogs(const Ticket& ticket, const ServerCallContext& context, + std::unique_ptr* result){}; + + /// \brief Returns a FlightInfo object which contains the data to access the + /// stream of data. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out] info The FlightInfo describing where to access the + /// dataset. + /// \return Status. + virtual Status GetFlightInfoSqlInfo(const pb::sql::CommandStatementQuery& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info){}; + + /// \brief Returns data for sqlInfo based on a data stream. + /// \param[in]command The sql command to generate the data stream. + /// \param[in]ticket The application-defined ticket identifying this stream. + /// \param[in]context Per-call context. + /// \param[out]result The Result iterator. + /// \return Status. + virtual Status DoGetSqlInfo(const pb::sql::CommandStatementQuery& command, + const Ticket& ticket, const ServerCallContext& context, + std::unique_ptr* result){}; + + /// \brief Returns a FlightInfo object which contains the data to access the + /// stream of data. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out] info The FlightInfo describing where to access the + /// dataset. + /// \return Status. + virtual Status GetFlightInfoSchemas(const pb::sql::CommandGetSchemas& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info){}; + + /// \brief Returns data for schemas based on a data stream. + /// \param[in]command The sql command to generate the data stream. + /// \param[in]ticket The application-defined ticket identifying this stream. + /// \param[in]context Per-call context. + /// \param[out]result The Result iterator. + /// \return Status. + virtual Status DoGetSchemas(const pb::sql::CommandGetSchemas& command, + const Ticket& ticket, const ServerCallContext& context, + std::unique_ptr* result){}; + + ///\brief Returns a FlightInfo object which contains the data to access the + /// stream of data. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out] info The FlightInfo describing where to access the + /// dataset. + /// \return Status. + virtual Status GetFlightInfoTables(const pb::sql::CommandGetTables& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info){}; + + /// \brief Returns data for tables based on a data stream. + /// \param[in]ticket The application-defined ticket identifying this stream. + /// \param[in]context Per-call context. + /// \param[out]result The Result iterator. + /// \return Status. + virtual Status DoGetTables(const pb::sql::CommandGetTables, const Ticket& ticket, + const ServerCallContext& context, + std::unique_ptr* result){}; + + /// \brief Returns a FlightInfo object which contains the data to access the + /// stream of data. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out] info The FlightInfo describing where to access the + /// dataset. + /// \return Status. + virtual Status GetFlightInfoTableTypes(const pb::sql::CommandGetTableTypes& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info){}; + + /// \brief Returns data for table types based on a data stream. + /// \param[in]ticket The application-defined ticket identifying this stream. + /// \param[in]context Per-call context. + /// \param[out]result The Result iterator. + /// \return Status. + virtual Status DoGetTablesTypes(const Ticket& ticket, const ServerCallContext& context, + std::unique_ptr* result){}; + + /// \brief Returns a FlightInfo object which contains the data to access the + /// stream of data. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out] info The FlightInfo describing where to access the + /// dataset. + /// \return Status. + virtual Status GetFlightInfoPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info){}; + + /// \brief Returns data for primary keys based on a data stream. + /// \param[in]command The sql command to generate the data stream. + /// \param[in]ticket The application-defined ticket identifying this stream. + /// \param[in]context Per-call context. + /// \param[out]result The Result iterator. + /// \return Status. + virtual Status DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, + const Ticket& ticket, const ServerCallContext& context, + std::unique_ptr* result){}; + + /// \brief Returns a FlightInfo object which contains the data to access the + /// stream of data. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out] info The FlightInfo describing where to access the + /// dataset. + /// \return Status. + virtual Status GetFlightInfoExportedKeys(const pb::sql::CommandGetExportedKeys& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info){}; + + /// \brief Returns data for exported keys based on a data stream. + /// \param[in]command The sql command to generate the data stream. + /// \param[in]ticket The application-defined ticket identifying this stream. + /// \param[in]context Per-call context. + /// \param[out]result The Result iterator. + /// \return Status. + virtual Status DoGetExportedKeys(const pb::sql::CommandGetExportedKeys& command, + const Ticket& ticket, const ServerCallContext& context, + std::unique_ptr* result){}; + + /// \brief Returns a FlightInfo object which contains the data to access the + /// stream of data. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out] info The FlightInfo describing where to access the + /// dataset. + /// \return Status. + virtual Status GetFlightInfoImportedKeys(const pb::sql::CommandGetImportedKeys& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info){}; + + /// \brief Returns data for imported keys based on a data stream. + /// \param[in]command The sql command to generate the data stream. + /// \param[in]ticket The application-defined ticket identifying this stream. + /// \param[in]context Per-call context. + /// \param[out]result The Result iterator. + /// \return Status. + virtual Status DoGetImportedKeys(const pb::sql::CommandGetImportedKeys& command, + const Ticket& ticket, const ServerCallContext& context, + std::unique_ptr* result){}; + + /// \brief Creates a prepared statement on the server and returns a handle and metadata + /// for in a + /// ActionCreatePreparedStatementResult object. + /// \param[in]command The sql command to generate the data stream. + /// \param[in]context Per-call context. + /// \param[out]result The Result iterator. + /// \return Status. + virtual Status CreatePreparedStatement( + const pb::sql::ActionCreatePreparedStatementRequest& command, + const ServerCallContext& context, std::unique_ptr* result){}; + + /// \brief Closes a prepared statement on the server. No result is expected. + /// \param[in]command The sql command to generate the data stream. + /// \param[in]context Per-call context. + /// \param[out]result The Result iterator. + /// \return Status. + virtual Status ClosePreparedStatement( + const pb::sql::ActionClosePreparedStatementRequest& command, + const ServerCallContext& context, std::unique_ptr* result){}; +}; +} // namespace sql +} // namespace flight +} // namespace arrow From eeb4a3531d607d9a57513a04ec98c5282e7cc681 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 22 Sep 2021 13:56:52 -0300 Subject: [PATCH 016/237] Remove flight-sql from CMakeLists.txt --- cpp/src/arrow/flight/flight-sql/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 3fe10183950..5cc000171ac 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -92,6 +92,6 @@ add_arrow_test(flight_sql_test LABELS "arrow_flight_sql") -add_executable(flight_sql_test_app test_app.cc sql_server.h) +add_executable(flight_sql_test_app test_app.cc) target_link_libraries(flight_sql_test_app PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES}) From a25bb904c90726473816ba9d6fc11be1441e111c Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 22 Sep 2021 14:00:27 -0300 Subject: [PATCH 017/237] Remove doPut* methods (not used yet) --- cpp/src/arrow/flight/flight-sql/sql_server.h | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index b2e27b16efc..c9120d21034 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -109,21 +109,6 @@ class FlightSqlServerBase : public FlightServerBase { const pb::sql::CommandPreparedStatementQuery& command, const Ticket& ticket, const ServerCallContext& context, std::unique_ptr* result){}; - // TODO check doPut - virtual Status DoPutStatement(const pb::sql::CommandStatementUpdate& command, - const ServerCallContext& context, - const FlightDescriptor& descriptor){}; - - // TODO check doPut - virtual Status DoPutPreparedStatementUpdate( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, const FlightDescriptor& descriptor){}; - - // TODO check doPut - virtual Status DoPutPreparedStatementQuery( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, const FlightDescriptor& descriptor){}; - /// \brief Returns a FlightInfo object which contains the data to access the /// stream of data. /// \param[in] command The sql command to generate the data stream. From c5d63016cf700fdd3bcd3526011608ac0482c652 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 22 Sep 2021 14:11:58 -0300 Subject: [PATCH 018/237] Add more statement to the getFlightInfo methods --- cpp/src/arrow/flight/flight-sql/sql_server.h | 41 ++++++++++++++++++-- 1 file changed, 37 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index c9120d21034..979a67b00b9 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -45,10 +45,43 @@ class FlightSqlServerBase : public FlightServerBase { if (any.Is()) { pb::sql::CommandStatementQuery command; any.UnpackTo(&command); - - GetFlightInfoStatement(command, context, request, info); - } else if (any.Is()) { - std::cout << "Update" << std::endl; + ARROW_RETURN_NOT_OK(GetFlightInfoStatement(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandPreparedStatementQuery command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoPreparedStatement(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetCatalogs command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoPreparedStatement(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetSchemas command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoCatalogs(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetTables command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoTables(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetTableTypes command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoTableTypes(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetSqlInfo command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoSqlInfo(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetPrimaryKeys command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoPrimaryKeys(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetExportedKeys command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoExportedKeys(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetImportedKeys command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoImportedKeys(command, context, request, info)); } return Status::OK(); From 5f7a4c81195c7bf688e1f82cef8781f7299d89d0 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 22 Sep 2021 14:13:36 -0300 Subject: [PATCH 019/237] Implement missing branches on DoGet --- cpp/src/arrow/flight/flight-sql/sql_server.h | 113 +++++++++++++------ 1 file changed, 77 insertions(+), 36 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 979a67b00b9..450e0fb544f 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -97,8 +97,52 @@ class FlightSqlServerBase : public FlightServerBase { if (anyCommand.Is()) { pb::sql::TicketStatementQuery command; anyCommand.UnpackTo(&command); - - DoGetStatement(command, context, request, stream); + ARROW_RETURN_NOT_OK(DoGetStatement(command, context, request, stream)); + } + if (anyCommand.Is()) { + pb::sql::CommandPreparedStatementQuery command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetPreparedStatement(command, context, request, payload)); + } + if (anyCommand.Is()) { + pb::sql::CommandGetCatalogs command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetCatalogs(command, context, request, payload)); + } + if (anyCommand.Is()) { + pb::sql::CommandGetSchemas command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetSchemas(command, context, request, payload)); + } + if (anyCommand.Is()) { + pb::sql::CommandGetTables command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetTables(command, context, request, payload)); + } + if (anyCommand.Is()) { + pb::sql::CommandGetTableTypes command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetTableTypes(command, context, request, payload)); + } + if (anyCommand.Is()) { + pb::sql::CommandGetSqlInfo command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetSqlInfo(command, context, request, payload)); + } + if (anyCommand.Is()) { + pb::sql::CommandGetPrimaryKeys command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetPrimaryKeys(command, context, request, payload)); + } + if (anyCommand.Is()) { + pb::sql::CommandGetExportedKeys command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetExportedKeys(command, context, request, payload)); + } + if (anyCommand.Is()) { + pb::sql::CommandGetImportedKeys command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetImportedKeys(command, context, request, payload)); } return Status::OK(); @@ -139,7 +183,7 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr* info){}; virtual Status DoGetPreparedStatement( - const pb::sql::CommandPreparedStatementQuery& command, const Ticket& ticket, + const pb::sql::CommandPreparedStatementQuery& command, const ServerCallContext& context, std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the @@ -156,11 +200,12 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr* info){}; /// \brief Returns data for catalogs based on a data stream. - /// \param[in] ticket The application-defined ticket identifying this stream. + /// \param[in] command The sql command to generate the data stream. /// \param[in] context Per-call context. /// \param[out] result An interface for sending data back to the client. /// \return Status. - virtual Status DoGetCatalogs(const Ticket& ticket, const ServerCallContext& context, + virtual Status DoGetCatalogs(const pb::sql::CommandGetCatalogs& command, + const ServerCallContext& context, std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the @@ -177,13 +222,12 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr* info){}; /// \brief Returns data for sqlInfo based on a data stream. - /// \param[in]command The sql command to generate the data stream. - /// \param[in]ticket The application-defined ticket identifying this stream. - /// \param[in]context Per-call context. - /// \param[out]result The Result iterator. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[out] result The Result iterator. /// \return Status. virtual Status DoGetSqlInfo(const pb::sql::CommandStatementQuery& command, - const Ticket& ticket, const ServerCallContext& context, + const ServerCallContext& context, std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the @@ -201,12 +245,11 @@ class FlightSqlServerBase : public FlightServerBase { /// \brief Returns data for schemas based on a data stream. /// \param[in]command The sql command to generate the data stream. - /// \param[in]ticket The application-defined ticket identifying this stream. /// \param[in]context Per-call context. /// \param[out]result The Result iterator. /// \return Status. virtual Status DoGetSchemas(const pb::sql::CommandGetSchemas& command, - const Ticket& ticket, const ServerCallContext& context, + const ServerCallContext& context, std::unique_ptr* result){}; ///\brief Returns a FlightInfo object which contains the data to access the @@ -223,11 +266,11 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr* info){}; /// \brief Returns data for tables based on a data stream. - /// \param[in]ticket The application-defined ticket identifying this stream. - /// \param[in]context Per-call context. - /// \param[out]result The Result iterator. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[out] result The Result iterator. /// \return Status. - virtual Status DoGetTables(const pb::sql::CommandGetTables, const Ticket& ticket, + virtual Status DoGetTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, std::unique_ptr* result){}; @@ -245,12 +288,13 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr* info){}; /// \brief Returns data for table types based on a data stream. - /// \param[in]ticket The application-defined ticket identifying this stream. - /// \param[in]context Per-call context. - /// \param[out]result The Result iterator. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[out] result The Result iterator. /// \return Status. - virtual Status DoGetTablesTypes(const Ticket& ticket, const ServerCallContext& context, - std::unique_ptr* result){}; + virtual Status DoGetTableTypes(const pb::sql::CommandGetTableTypes& command, + const ServerCallContext& context, + std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the /// stream of data. @@ -266,13 +310,12 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr* info){}; /// \brief Returns data for primary keys based on a data stream. - /// \param[in]command The sql command to generate the data stream. - /// \param[in]ticket The application-defined ticket identifying this stream. - /// \param[in]context Per-call context. - /// \param[out]result The Result iterator. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[out] result The Result iterator. /// \return Status. virtual Status DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, - const Ticket& ticket, const ServerCallContext& context, + const ServerCallContext& context, std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the @@ -289,13 +332,12 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr* info){}; /// \brief Returns data for exported keys based on a data stream. - /// \param[in]command The sql command to generate the data stream. - /// \param[in]ticket The application-defined ticket identifying this stream. - /// \param[in]context Per-call context. - /// \param[out]result The Result iterator. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[out] result The Result iterator. /// \return Status. virtual Status DoGetExportedKeys(const pb::sql::CommandGetExportedKeys& command, - const Ticket& ticket, const ServerCallContext& context, + const ServerCallContext& context, std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the @@ -312,13 +354,12 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr* info){}; /// \brief Returns data for imported keys based on a data stream. - /// \param[in]command The sql command to generate the data stream. - /// \param[in]ticket The application-defined ticket identifying this stream. - /// \param[in]context Per-call context. - /// \param[out]result The Result iterator. + /// \param[in] command The sql command to generate the data stream. + /// \param[in] context Per-call context. + /// \param[out] result The Result iterator. /// \return Status. virtual Status DoGetImportedKeys(const pb::sql::CommandGetImportedKeys& command, - const Ticket& ticket, const ServerCallContext& context, + const ServerCallContext& context, std::unique_ptr* result){}; /// \brief Creates a prepared statement on the server and returns a handle and metadata From 8aaee1edcf1e248bb62ec4c81e0ef52097879ec0 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 22 Sep 2021 14:29:16 -0300 Subject: [PATCH 020/237] Fix wrong arguments on server header file --- cpp/src/arrow/flight/flight-sql/sql_server.h | 52 ++++++++++---------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 450e0fb544f..d582105529b 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -53,11 +53,11 @@ class FlightSqlServerBase : public FlightServerBase { } else if (any.Is()) { pb::sql::CommandGetCatalogs command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoPreparedStatement(command, context, request, info)); + ARROW_RETURN_NOT_OK(GetFlightInfoCatalogs(command, context, request, info)); } else if (any.Is()) { pb::sql::CommandGetSchemas command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoCatalogs(command, context, request, info)); + ARROW_RETURN_NOT_OK(GetFlightInfoSchemas(command, context, request, info)); } else if (any.Is()) { pb::sql::CommandGetTables command; any.UnpackTo(&command); @@ -97,52 +97,52 @@ class FlightSqlServerBase : public FlightServerBase { if (anyCommand.Is()) { pb::sql::TicketStatementQuery command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetStatement(command, context, request, stream)); + ARROW_RETURN_NOT_OK(DoGetStatement(command, context, stream)); } if (anyCommand.Is()) { pb::sql::CommandPreparedStatementQuery command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetPreparedStatement(command, context, request, payload)); + ARROW_RETURN_NOT_OK(DoGetPreparedStatement(command, context, stream)); } if (anyCommand.Is()) { pb::sql::CommandGetCatalogs command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetCatalogs(command, context, request, payload)); + ARROW_RETURN_NOT_OK(DoGetCatalogs(command, context, stream)); } if (anyCommand.Is()) { pb::sql::CommandGetSchemas command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetSchemas(command, context, request, payload)); + ARROW_RETURN_NOT_OK(DoGetSchemas(command, context, stream)); } if (anyCommand.Is()) { pb::sql::CommandGetTables command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetTables(command, context, request, payload)); + ARROW_RETURN_NOT_OK(DoGetTables(command, context, stream)); } if (anyCommand.Is()) { pb::sql::CommandGetTableTypes command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetTableTypes(command, context, request, payload)); + ARROW_RETURN_NOT_OK(DoGetTableTypes(command, context, stream)); } if (anyCommand.Is()) { pb::sql::CommandGetSqlInfo command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetSqlInfo(command, context, request, payload)); + ARROW_RETURN_NOT_OK(DoGetSqlInfo(command, context, stream)); } if (anyCommand.Is()) { pb::sql::CommandGetPrimaryKeys command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetPrimaryKeys(command, context, request, payload)); + ARROW_RETURN_NOT_OK(DoGetPrimaryKeys(command, context, stream)); } if (anyCommand.Is()) { pb::sql::CommandGetExportedKeys command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetExportedKeys(command, context, request, payload)); + ARROW_RETURN_NOT_OK(DoGetExportedKeys(command, context, stream)); } if (anyCommand.Is()) { pb::sql::CommandGetImportedKeys command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetImportedKeys(command, context, request, payload)); + ARROW_RETURN_NOT_OK(DoGetImportedKeys(command, context, stream)); } return Status::OK(); @@ -167,7 +167,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \param[out]result The Result iterator. /// \return Status. virtual Status DoGetStatement(const pb::sql::TicketStatementQuery& command, - const ServerCallContext& context, const Ticket& ticket, + const ServerCallContext& context, std::unique_ptr* result){}; /// \brief Gets information about a particular prepared statement data stream. @@ -184,7 +184,7 @@ class FlightSqlServerBase : public FlightServerBase { virtual Status DoGetPreparedStatement( const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, std::unique_ptr* result){}; + const ServerCallContext& context, std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the /// stream of data. @@ -206,7 +206,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetCatalogs(const pb::sql::CommandGetCatalogs& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the /// stream of data. @@ -216,7 +216,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoSqlInfo(const pb::sql::CommandStatementQuery& command, + virtual Status GetFlightInfoSqlInfo(const pb::sql::CommandGetSqlInfo& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info){}; @@ -226,9 +226,9 @@ class FlightSqlServerBase : public FlightServerBase { /// \param[in] context Per-call context. /// \param[out] result The Result iterator. /// \return Status. - virtual Status DoGetSqlInfo(const pb::sql::CommandStatementQuery& command, + virtual Status DoGetSqlInfo(const pb::sql::CommandGetSqlInfo& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the /// stream of data. @@ -250,7 +250,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetSchemas(const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result){}; ///\brief Returns a FlightInfo object which contains the data to access the /// stream of data. @@ -272,7 +272,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the /// stream of data. @@ -294,7 +294,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetTableTypes(const pb::sql::CommandGetTableTypes& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the /// stream of data. @@ -316,7 +316,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the /// stream of data. @@ -338,7 +338,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetExportedKeys(const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result){}; /// \brief Returns a FlightInfo object which contains the data to access the /// stream of data. @@ -360,7 +360,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetImportedKeys(const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result){}; /// \brief Creates a prepared statement on the server and returns a handle and metadata /// for in a @@ -371,7 +371,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status CreatePreparedStatement( const pb::sql::ActionCreatePreparedStatementRequest& command, - const ServerCallContext& context, std::unique_ptr* result){}; + const ServerCallContext& context, std::unique_ptr* result){}; /// \brief Closes a prepared statement on the server. No result is expected. /// \param[in]command The sql command to generate the data stream. @@ -380,7 +380,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status ClosePreparedStatement( const pb::sql::ActionClosePreparedStatementRequest& command, - const ServerCallContext& context, std::unique_ptr* result){}; + const ServerCallContext& context, std::unique_ptr* result){}; }; } // namespace sql } // namespace flight From 4099ad21518a087ed818587be3e73ef84368398e Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 22 Sep 2021 15:16:52 -0300 Subject: [PATCH 021/237] Improve documentation on sql_server.h --- cpp/src/arrow/flight/flight-sql/sql_server.h | 190 +++++++++---------- 1 file changed, 87 insertions(+), 103 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index d582105529b..24a815153e8 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -49,7 +49,8 @@ class FlightSqlServerBase : public FlightServerBase { } else if (any.Is()) { pb::sql::CommandPreparedStatementQuery command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoPreparedStatement(command, context, request, info)); + ARROW_RETURN_NOT_OK( + GetFlightInfoPreparedStatement(command, context, request, info)); } else if (any.Is()) { pb::sql::CommandGetCatalogs command; any.UnpackTo(&command); @@ -62,7 +63,7 @@ class FlightSqlServerBase : public FlightServerBase { pb::sql::CommandGetTables command; any.UnpackTo(&command); ARROW_RETURN_NOT_OK(GetFlightInfoTables(command, context, request, info)); - } else if (any.Is()) { + } else if (any.Is()) { pb::sql::CommandGetTableTypes command; any.UnpackTo(&command); ARROW_RETURN_NOT_OK(GetFlightInfoTableTypes(command, context, request, info)); @@ -148,30 +149,31 @@ class FlightSqlServerBase : public FlightServerBase { return Status::OK(); } - /// \brief Gets information about a particular SQL query based data stream. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightInfo for executing a SQL query. + /// \param[in] command The CommandStatementQuery object containing the SQL + /// statement. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. - /// \param[out] info The FlightInfo describing where to access the - /// dataset. + /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. virtual Status GetFlightInfoStatement(const pb::sql::CommandStatementQuery& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info){}; - /// \brief Returns data for a SQL query based data stream. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightDataStream containing the query results. + /// \param[in] command The TicketStatementQuery containing the statement handle. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. - /// \param[out]result The Result iterator. + /// \param[out] result The FlightDataStream containing the results. /// \return Status. virtual Status DoGetStatement(const pb::sql::TicketStatementQuery& command, const ServerCallContext& context, std::unique_ptr* result){}; - /// \brief Gets information about a particular prepared statement data stream. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightInfo for executing an already created prepared statement. + /// \param[in] command The CommandPreparedStatementQuery object containing the + /// prepared statement handle. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the @@ -182,123 +184,119 @@ class FlightSqlServerBase : public FlightServerBase { const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info){}; + /// \brief Gets a FlightDataStream containing the prepared statement query results. + /// \param[in] command The CommandPreparedStatementQuery object containing the + /// prepared statement handle. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out] result The FlightDataStream containing the results. + /// \return Status. virtual Status DoGetPreparedStatement( const pb::sql::CommandPreparedStatementQuery& command, const ServerCallContext& context, std::unique_ptr* result){}; - /// \brief Returns a FlightInfo object which contains the data to access the - /// stream of data. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightInfo for listing catalogs. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. - /// \param[out] info The FlightInfo describing where to access the - /// dataset. + /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status GetFlightInfoCatalogs(const pb::sql::CommandGetCatalogs& command, - const ServerCallContext& context, + virtual Status GetFlightInfoCatalogs(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info){}; - /// \brief Returns data for catalogs based on a data stream. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightDataStream containing the list of catalogs. /// \param[in] context Per-call context. /// \param[out] result An interface for sending data back to the client. /// \return Status. - virtual Status DoGetCatalogs(const pb::sql::CommandGetCatalogs& command, - const ServerCallContext& context, + virtual Status DoGetCatalogs(const ServerCallContext& context, std::unique_ptr* result){}; - /// \brief Returns a FlightInfo object which contains the data to access the - /// stream of data. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightInfo for retrieving other information (See SqlInfo). + /// \param[in] command The CommandGetSqlInfo object containing the list of SqlInfo + /// to be returned. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. - /// \param[out] info The FlightInfo describing where to access the - /// dataset. + /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. virtual Status GetFlightInfoSqlInfo(const pb::sql::CommandGetSqlInfo& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info){}; - /// \brief Returns data for sqlInfo based on a data stream. - /// \param[in] command The sql command to generate the data stream. - /// \param[in] context Per-call context. - /// \param[out] result The Result iterator. - /// \return Status. + /// \brief Gets a FlightDataStream containing the list of SqlInfo results. + /// \param[in] command The CommandGetSqlInfo object containing the list of SqlInfo + /// to be returned. + /// \param[in] context Per-call context. + /// \param[out] result The FlightDataStream containing the results. + /// \return Status. virtual Status DoGetSqlInfo(const pb::sql::CommandGetSqlInfo& command, const ServerCallContext& context, std::unique_ptr* result){}; - /// \brief Returns a FlightInfo object which contains the data to access the - /// stream of data. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightInfo for listing schemas. + /// \param[in] command The CommandGetSchemas object which may contain filters for + /// catalog and schema name. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. - /// \param[out] info The FlightInfo describing where to access the - /// dataset. + /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. virtual Status GetFlightInfoSchemas(const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info){}; - /// \brief Returns data for schemas based on a data stream. - /// \param[in]command The sql command to generate the data stream. - /// \param[in]context Per-call context. - /// \param[out]result The Result iterator. - /// \return Status. + /// \brief Gets a FlightDataStream containing the list of schemas. + /// \param[in] command The CommandGetSchemas object which may contain filters for + /// catalog and schema name. + /// \param[in] context Per-call context. + /// \param[out] result The FlightDataStream containing the results. + /// \return Status. virtual Status DoGetSchemas(const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, std::unique_ptr* result){}; - ///\brief Returns a FlightInfo object which contains the data to access the - /// stream of data. - /// \param[in] command The sql command to generate the data stream. + ///\brief Gets a FlightInfo for listing tables. + /// \param[in] command The CommandGetTables object which may contain filters for + /// catalog, schema and table names. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. - /// \param[out] info The FlightInfo describing where to access the - /// dataset. + /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. virtual Status GetFlightInfoTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info){}; - /// \brief Returns data for tables based on a data stream. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightDataStream containing the list of tables. + /// \param[in] command The CommandGetTables object which may contain filters for + /// catalog, schema and table names. /// \param[in] context Per-call context. - /// \param[out] result The Result iterator. - /// \return Status. + /// \param[out] result The FlightDataStream containing the results. + /// \return Status. virtual Status DoGetTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, std::unique_ptr* result){}; - /// \brief Returns a FlightInfo object which contains the data to access the - /// stream of data. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightInfo to extract information about the table types. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoTableTypes(const pb::sql::CommandGetTableTypes& command, - const ServerCallContext& context, + virtual Status GetFlightInfoTableTypes(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info){}; - /// \brief Returns data for table types based on a data stream. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightDataStream containing the data related to the table types. /// \param[in] context Per-call context. - /// \param[out] result The Result iterator. - /// \return Status. - virtual Status DoGetTableTypes(const pb::sql::CommandGetTableTypes& command, - const ServerCallContext& context, + /// \param[out] result The FlightDataStream containing the results. + /// \return Status. + virtual Status DoGetTableTypes(const ServerCallContext& context, std::unique_ptr* result){}; - /// \brief Returns a FlightInfo object which contains the data to access the - /// stream of data. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightInfo to extract information about primary and foreign keys. + /// \param[in] command The CommandGetPrimaryKeys object with necessary information + /// to execute the request. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the @@ -309,18 +307,20 @@ class FlightSqlServerBase : public FlightServerBase { const FlightDescriptor& descriptor, std::unique_ptr* info){}; - /// \brief Returns data for primary keys based on a data stream. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightDataStream containing the data related to the primary and foreign + /// keys. + /// \param[in] command The CommandGetPrimaryKeys object with necessary information + /// to execute the request. /// \param[in] context Per-call context. - /// \param[out] result The Result iterator. - /// \return Status. + /// \param[out] result The FlightDataStream containing the results. + /// \return Status. virtual Status DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, std::unique_ptr* result){}; - /// \brief Returns a FlightInfo object which contains the data to access the - /// stream of data. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightInfo to extract information about foreign and primary keys. + /// \param[in] command The CommandGetExportedKeys object with necessary information + /// to execute the request. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the @@ -331,18 +331,20 @@ class FlightSqlServerBase : public FlightServerBase { const FlightDescriptor& descriptor, std::unique_ptr* info){}; - /// \brief Returns data for exported keys based on a data stream. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightDataStream containing the data related to the foreign and primary + /// keys. + /// \param[in] command The CommandGetExportedKeys object with necessary information + /// to execute the request. /// \param[in] context Per-call context. - /// \param[out] result The Result iterator. - /// \return Status. + /// \param[out] result The FlightDataStream containing the results. + /// \return Status. virtual Status DoGetExportedKeys(const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, std::unique_ptr* result){}; - /// \brief Returns a FlightInfo object which contains the data to access the - /// stream of data. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightInfo to extract information about foreign and primary keys. + /// \param[in] command The CommandGetImportedKeys object with necessary information + /// to execute the request. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the @@ -353,34 +355,16 @@ class FlightSqlServerBase : public FlightServerBase { const FlightDescriptor& descriptor, std::unique_ptr* info){}; - /// \brief Returns data for imported keys based on a data stream. - /// \param[in] command The sql command to generate the data stream. + /// \brief Gets a FlightDataStream containing the data related to the foreign and + /// primary keys. + /// \param[in] command The CommandGetImportedKeys object with necessary information + /// to execute the request. /// \param[in] context Per-call context. - /// \param[out] result The Result iterator. - /// \return Status. + /// \param[out] result The FlightDataStream containing the results. + /// \return Status. virtual Status DoGetImportedKeys(const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, std::unique_ptr* result){}; - - /// \brief Creates a prepared statement on the server and returns a handle and metadata - /// for in a - /// ActionCreatePreparedStatementResult object. - /// \param[in]command The sql command to generate the data stream. - /// \param[in]context Per-call context. - /// \param[out]result The Result iterator. - /// \return Status. - virtual Status CreatePreparedStatement( - const pb::sql::ActionCreatePreparedStatementRequest& command, - const ServerCallContext& context, std::unique_ptr* result){}; - - /// \brief Closes a prepared statement on the server. No result is expected. - /// \param[in]command The sql command to generate the data stream. - /// \param[in]context Per-call context. - /// \param[out]result The Result iterator. - /// \return Status. - virtual Status ClosePreparedStatement( - const pb::sql::ActionClosePreparedStatementRequest& command, - const ServerCallContext& context, std::unique_ptr* result){}; }; } // namespace sql } // namespace flight From 685ccd00406593a35ee4156004db5a4b9a499c03 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 23 Sep 2021 11:38:27 -0300 Subject: [PATCH 022/237] Add missing else if on DoGet method --- cpp/src/arrow/flight/flight-sql/sql_server.h | 27 +++++++------------- 1 file changed, 9 insertions(+), 18 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 24a815153e8..2f4bf6f4757 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -99,48 +99,39 @@ class FlightSqlServerBase : public FlightServerBase { pb::sql::TicketStatementQuery command; anyCommand.UnpackTo(&command); ARROW_RETURN_NOT_OK(DoGetStatement(command, context, stream)); - } - if (anyCommand.Is()) { + } else if (anyCommand.Is()) { pb::sql::CommandPreparedStatementQuery command; anyCommand.UnpackTo(&command); ARROW_RETURN_NOT_OK(DoGetPreparedStatement(command, context, stream)); - } - if (anyCommand.Is()) { + } else if (anyCommand.Is()) { pb::sql::CommandGetCatalogs command; anyCommand.UnpackTo(&command); ARROW_RETURN_NOT_OK(DoGetCatalogs(command, context, stream)); - } - if (anyCommand.Is()) { + } else if (anyCommand.Is()) { pb::sql::CommandGetSchemas command; anyCommand.UnpackTo(&command); ARROW_RETURN_NOT_OK(DoGetSchemas(command, context, stream)); - } - if (anyCommand.Is()) { + } else if (anyCommand.Is()) { pb::sql::CommandGetTables command; anyCommand.UnpackTo(&command); ARROW_RETURN_NOT_OK(DoGetTables(command, context, stream)); - } - if (anyCommand.Is()) { + } else if (anyCommand.Is()) { pb::sql::CommandGetTableTypes command; anyCommand.UnpackTo(&command); ARROW_RETURN_NOT_OK(DoGetTableTypes(command, context, stream)); - } - if (anyCommand.Is()) { + } else if (anyCommand.Is()) { pb::sql::CommandGetSqlInfo command; anyCommand.UnpackTo(&command); ARROW_RETURN_NOT_OK(DoGetSqlInfo(command, context, stream)); - } - if (anyCommand.Is()) { + } else if (anyCommand.Is()) { pb::sql::CommandGetPrimaryKeys command; anyCommand.UnpackTo(&command); ARROW_RETURN_NOT_OK(DoGetPrimaryKeys(command, context, stream)); - } - if (anyCommand.Is()) { + } else if (anyCommand.Is()) { pb::sql::CommandGetExportedKeys command; anyCommand.UnpackTo(&command); ARROW_RETURN_NOT_OK(DoGetExportedKeys(command, context, stream)); - } - if (anyCommand.Is()) { + } else if (anyCommand.Is()) { pb::sql::CommandGetImportedKeys command; anyCommand.UnpackTo(&command); ARROW_RETURN_NOT_OK(DoGetImportedKeys(command, context, stream)); From cffd1f4cbb5d9199bb47807e7fef131ac4359a7a Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 24 Sep 2021 15:20:36 -0300 Subject: [PATCH 023/237] Separate implementation from header file --- .../arrow/flight/flight-sql/sql_server.cpp | 251 ++++++++++++++++++ cpp/src/arrow/flight/flight-sql/sql_server.h | 151 ++--------- 2 files changed, 279 insertions(+), 123 deletions(-) create mode 100644 cpp/src/arrow/flight/flight-sql/sql_server.cpp diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp new file mode 100644 index 00000000000..a4181669f79 --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -0,0 +1,251 @@ +// 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/flight-sql/sql_server.h" + +namespace arrow { +namespace flight { +namespace sql { +Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, + const FlightDescriptor& request, + std::unique_ptr* info) { + google::protobuf::Any any; + any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size())); + + if (any.Is()) { + pb::sql::CommandStatementQuery command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoStatement(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandPreparedStatementQuery command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoPreparedStatement(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetCatalogs command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoCatalogs(context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetSchemas command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoSchemas(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetTables command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoTables(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetTableTypes command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoTableTypes(context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetSqlInfo command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoSqlInfo(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetPrimaryKeys command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoPrimaryKeys(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetExportedKeys command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoExportedKeys(command, context, request, info)); + } else if (any.Is()) { + pb::sql::CommandGetImportedKeys command; + any.UnpackTo(&command); + ARROW_RETURN_NOT_OK(GetFlightInfoImportedKeys(command, context, request, info)); + } + + return Status::OK(); +} + +Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket& request, + std::unique_ptr* stream) { + google::protobuf::Any anyCommand; + + anyCommand.ParseFromArray(request.ticket.data(), + static_cast(request.ticket.size())); + + if (anyCommand.Is()) { + pb::sql::TicketStatementQuery command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetStatement(command, context, stream)); + } else if (anyCommand.Is()) { + pb::sql::CommandPreparedStatementQuery command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetPreparedStatement(command, context, stream)); + } else if (anyCommand.Is()) { + pb::sql::CommandGetCatalogs command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetCatalogs(context, stream)); + } else if (anyCommand.Is()) { + pb::sql::CommandGetSchemas command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetSchemas(command, context, stream)); + } else if (anyCommand.Is()) { + pb::sql::CommandGetTables command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetTables(command, context, stream)); + } else if (anyCommand.Is()) { + pb::sql::CommandGetTableTypes command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetTableTypes(context, stream)); + } else if (anyCommand.Is()) { + pb::sql::CommandGetSqlInfo command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetSqlInfo(command, context, stream)); + } else if (anyCommand.Is()) { + pb::sql::CommandGetPrimaryKeys command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetPrimaryKeys(command, context, stream)); + } else if (anyCommand.Is()) { + pb::sql::CommandGetExportedKeys command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetExportedKeys(command, context, stream)); + } else if (anyCommand.Is()) { + pb::sql::CommandGetImportedKeys command; + anyCommand.UnpackTo(&command); + ARROW_RETURN_NOT_OK(DoGetImportedKeys(command, context, stream)); + } + + return Status::OK(); +} + +Status FlightSqlServerBase::GetFlightInfoCatalogs(const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { + return Status::NotImplemented("GetFlightInfoCatalogs not Implemented"); +} + +Status FlightSqlServerBase::DoGetImportedKeys( + const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, + std::unique_ptr* result) { + return Status::NotImplemented("DoGetImportedKeys not implemented"); +} + +Status FlightSqlServerBase::GetFlightInfoStatement( + const pb::sql::CommandStatementQuery& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return Status::NotImplemented("DoGetImportedKeys not implemented"); +} + +Status FlightSqlServerBase::DoGetStatement(const pb::sql::TicketStatementQuery& command, + const ServerCallContext& context, + std::unique_ptr* result) { + return Status::NotImplemented("GetFlightInfoStatement not implemented"); +} + +Status FlightSqlServerBase::GetFlightInfoPreparedStatement( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, const FlightDescriptor& descriptor, + std::unique_ptr* info) { + return Status::NotImplemented("DoGetStatement not implemented"); +} + +Status FlightSqlServerBase::DoGetPreparedStatement( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, std::unique_ptr* result) { + return Status::NotImplemented("GetFlightInfoPreparedStatement not implemented"); +} + +Status FlightSqlServerBase::DoGetCatalogs(const ServerCallContext& context, + std::unique_ptr* result) { + return Status::NotImplemented("DoGetCatalogs not implemented"); +} + +Status FlightSqlServerBase::GetFlightInfoSqlInfo( + const pb::sql::CommandGetSqlInfo& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return Status::NotImplemented("DoGetPreparedStatement not implemented"); +} + +Status FlightSqlServerBase::DoGetSqlInfo(const pb::sql::CommandGetSqlInfo& command, + const ServerCallContext& context, + std::unique_ptr* result) { + return Status::NotImplemented("GetFlightInfoSqlInfo not implemented"); +} + +Status FlightSqlServerBase::GetFlightInfoSchemas( + const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return Status::NotImplemented("DoGetSqlInfo not implemented"); +} + +Status FlightSqlServerBase::DoGetSchemas(const pb::sql::CommandGetSchemas& command, + const ServerCallContext& context, + std::unique_ptr* result) { + return Status::NotImplemented("GetFlightInfoSchemas not implemented"); +} + +Status FlightSqlServerBase::GetFlightInfoTables(const pb::sql::CommandGetTables& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { + return Status::NotImplemented("DoGetSchemas not implemented"); +} + +Status FlightSqlServerBase::DoGetTables(const pb::sql::CommandGetTables& command, + const ServerCallContext& context, + std::unique_ptr* result) { + return Status::NotImplemented("GetFlightInfoTables not implemented"); +} + +Status FlightSqlServerBase::GetFlightInfoTableTypes(const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { + return Status::NotImplemented("DoGetTables not implemented"); +} + +Status FlightSqlServerBase::DoGetTableTypes(const ServerCallContext& context, + std::unique_ptr* result) { + return Status::NotImplemented("DoGetTableTypes not implemented"); +} + +Status FlightSqlServerBase::GetFlightInfoPrimaryKeys( + const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return Status::NotImplemented("GetFlightInfoTableTypes not implemented"); +} + +Status FlightSqlServerBase::DoGetPrimaryKeys( + const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, + std::unique_ptr* result) { + return Status::NotImplemented("GetFlightInfoPrimaryKeys not implemented"); +} + +Status FlightSqlServerBase::GetFlightInfoExportedKeys( + const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return Status::NotImplemented("DoGetPrimaryKeys not implemented"); +} + +Status FlightSqlServerBase::DoGetExportedKeys( + const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, + std::unique_ptr* result) { + return Status::NotImplemented("GetFlightInfoExportedKeys not implemented"); +} + +Status FlightSqlServerBase::GetFlightInfoImportedKeys( + const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return Status::NotImplemented("DoGetExportedKeys not implemented"); +} + +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 2f4bf6f4757..95a98b55f2b 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -33,112 +33,15 @@ namespace sql { class FlightSqlServerBase : public FlightServerBase { public: - FlightSqlServerBase(){}; + FlightSqlServerBase(); ~FlightSqlServerBase() = default; Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, - std::unique_ptr* info) override { - google::protobuf::Any any; - any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size())); - - if (any.Is()) { - pb::sql::CommandStatementQuery command; - any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoStatement(command, context, request, info)); - } else if (any.Is()) { - pb::sql::CommandPreparedStatementQuery command; - any.UnpackTo(&command); - ARROW_RETURN_NOT_OK( - GetFlightInfoPreparedStatement(command, context, request, info)); - } else if (any.Is()) { - pb::sql::CommandGetCatalogs command; - any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoCatalogs(command, context, request, info)); - } else if (any.Is()) { - pb::sql::CommandGetSchemas command; - any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoSchemas(command, context, request, info)); - } else if (any.Is()) { - pb::sql::CommandGetTables command; - any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoTables(command, context, request, info)); - } else if (any.Is()) { - pb::sql::CommandGetTableTypes command; - any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoTableTypes(command, context, request, info)); - } else if (any.Is()) { - pb::sql::CommandGetSqlInfo command; - any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoSqlInfo(command, context, request, info)); - } else if (any.Is()) { - pb::sql::CommandGetPrimaryKeys command; - any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoPrimaryKeys(command, context, request, info)); - } else if (any.Is()) { - pb::sql::CommandGetExportedKeys command; - any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoExportedKeys(command, context, request, info)); - } else if (any.Is()) { - pb::sql::CommandGetImportedKeys command; - any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoImportedKeys(command, context, request, info)); - } - - return Status::OK(); - }; + std::unique_ptr* info) override; Status DoGet(const ServerCallContext& context, const Ticket& request, - std::unique_ptr* stream) override { - google::protobuf::Any anyCommand; - - anyCommand.ParseFromArray(request.ticket.data(), - static_cast(request.ticket.size())); - - if (anyCommand.Is()) { - pb::sql::TicketStatementQuery command; - anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetStatement(command, context, stream)); - } else if (anyCommand.Is()) { - pb::sql::CommandPreparedStatementQuery command; - anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetPreparedStatement(command, context, stream)); - } else if (anyCommand.Is()) { - pb::sql::CommandGetCatalogs command; - anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetCatalogs(command, context, stream)); - } else if (anyCommand.Is()) { - pb::sql::CommandGetSchemas command; - anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetSchemas(command, context, stream)); - } else if (anyCommand.Is()) { - pb::sql::CommandGetTables command; - anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetTables(command, context, stream)); - } else if (anyCommand.Is()) { - pb::sql::CommandGetTableTypes command; - anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetTableTypes(command, context, stream)); - } else if (anyCommand.Is()) { - pb::sql::CommandGetSqlInfo command; - anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetSqlInfo(command, context, stream)); - } else if (anyCommand.Is()) { - pb::sql::CommandGetPrimaryKeys command; - anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetPrimaryKeys(command, context, stream)); - } else if (anyCommand.Is()) { - pb::sql::CommandGetExportedKeys command; - anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetExportedKeys(command, context, stream)); - } else if (anyCommand.Is()) { - pb::sql::CommandGetImportedKeys command; - anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetImportedKeys(command, context, stream)); - } - - return Status::OK(); - } + std::unique_ptr* stream) override; /// \brief Gets a FlightInfo for executing a SQL query. /// \param[in] command The CommandStatementQuery object containing the SQL @@ -150,7 +53,7 @@ class FlightSqlServerBase : public FlightServerBase { virtual Status GetFlightInfoStatement(const pb::sql::CommandStatementQuery& command, const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info){}; + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the query results. /// \param[in] command The TicketStatementQuery containing the statement handle. @@ -160,12 +63,12 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetStatement(const pb::sql::TicketStatementQuery& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result); /// \brief Gets a FlightInfo for executing an already created prepared statement. /// \param[in] command The CommandPreparedStatementQuery object containing the /// prepared statement handle. - /// \param[in] context Per-call context. + /// \param[in] context Per-callcommand context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. @@ -173,7 +76,7 @@ class FlightSqlServerBase : public FlightServerBase { virtual Status GetFlightInfoPreparedStatement( const pb::sql::CommandPreparedStatementQuery& command, const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info){}; + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the prepared statement query results. /// \param[in] command The CommandPreparedStatementQuery object containing the @@ -184,7 +87,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetPreparedStatement( const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, std::unique_ptr* result){}; + const ServerCallContext& context, std::unique_ptr* result); /// \brief Gets a FlightInfo for listing catalogs. /// \param[in] context Per-call context. @@ -193,14 +96,14 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status GetFlightInfoCatalogs(const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info){}; + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the list of catalogs. /// \param[in] context Per-call context. /// \param[out] result An interface for sending data back to the client. /// \return Status. virtual Status DoGetCatalogs(const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result); /// \brief Gets a FlightInfo for retrieving other information (See SqlInfo). /// \param[in] command The CommandGetSqlInfo object containing the list of SqlInfo @@ -212,7 +115,7 @@ class FlightSqlServerBase : public FlightServerBase { virtual Status GetFlightInfoSqlInfo(const pb::sql::CommandGetSqlInfo& command, const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info){}; + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the list of SqlInfo results. /// \param[in] command The CommandGetSqlInfo object containing the list of SqlInfo @@ -222,7 +125,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetSqlInfo(const pb::sql::CommandGetSqlInfo& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result); /// \brief Gets a FlightInfo for listing schemas. /// \param[in] command The CommandGetSchemas object which may contain filters for @@ -234,7 +137,7 @@ class FlightSqlServerBase : public FlightServerBase { virtual Status GetFlightInfoSchemas(const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info){}; + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the list of schemas. /// \param[in] command The CommandGetSchemas object which may contain filters for @@ -244,7 +147,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetSchemas(const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result); ///\brief Gets a FlightInfo for listing tables. /// \param[in] command The CommandGetTables object which may contain filters for @@ -256,7 +159,7 @@ class FlightSqlServerBase : public FlightServerBase { virtual Status GetFlightInfoTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info){}; + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the list of tables. /// \param[in] command The CommandGetTables object which may contain filters for @@ -266,7 +169,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about the table types. /// \param[in] context Per-call context. @@ -276,14 +179,14 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status GetFlightInfoTableTypes(const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info){}; + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the data related to the table types. /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. virtual Status DoGetTableTypes(const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about primary and foreign keys. /// \param[in] command The CommandGetPrimaryKeys object with necessary information @@ -296,9 +199,10 @@ class FlightSqlServerBase : public FlightServerBase { virtual Status GetFlightInfoPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info){}; + std::unique_ptr* info); - /// \brief Gets a FlightDataStream containing the data related to the primary and foreign + /// \brief Gets a FlightDataStream containing the data related to the primary and + /// foreign /// keys. /// \param[in] command The CommandGetPrimaryKeys object with necessary information /// to execute the request. @@ -307,7 +211,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about foreign and primary keys. /// \param[in] command The CommandGetExportedKeys object with necessary information @@ -320,9 +224,10 @@ class FlightSqlServerBase : public FlightServerBase { virtual Status GetFlightInfoExportedKeys(const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info){}; + std::unique_ptr* info); - /// \brief Gets a FlightDataStream containing the data related to the foreign and primary + /// \brief Gets a FlightDataStream containing the data related to the foreign and + /// primary /// keys. /// \param[in] command The CommandGetExportedKeys object with necessary information /// to execute the request. @@ -331,7 +236,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetExportedKeys(const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about foreign and primary keys. /// \param[in] command The CommandGetImportedKeys object with necessary information @@ -344,7 +249,7 @@ class FlightSqlServerBase : public FlightServerBase { virtual Status GetFlightInfoImportedKeys(const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info){}; + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the data related to the foreign and /// primary keys. @@ -355,7 +260,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \return Status. virtual Status DoGetImportedKeys(const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, - std::unique_ptr* result){}; + std::unique_ptr* result); }; } // namespace sql } // namespace flight From 5bd548fb8f720397c1e7756151ac8c68190741c2 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 24 Sep 2021 15:35:25 -0300 Subject: [PATCH 024/237] Remove empty constructor and destructor from FlightSqlServerBase header --- cpp/src/arrow/flight/flight-sql/sql_server.h | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 95a98b55f2b..ab2fc787ed4 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -33,10 +33,6 @@ namespace sql { class FlightSqlServerBase : public FlightServerBase { public: - FlightSqlServerBase(); - - ~FlightSqlServerBase() = default; - Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, std::unique_ptr* info) override; @@ -68,7 +64,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \brief Gets a FlightInfo for executing an already created prepared statement. /// \param[in] command The CommandPreparedStatementQuery object containing the /// prepared statement handle. - /// \param[in] context Per-callcommand context. + /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. From 9dd9a3216c735183039cd87fb9d077433fec17f1 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 28 Sep 2021 14:00:22 -0300 Subject: [PATCH 025/237] Change default Status return on GetFlightInfo and DoGet --- cpp/src/arrow/flight/flight-sql/sql_server.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index a4181669f79..d72c840c336 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -71,7 +71,7 @@ Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, ARROW_RETURN_NOT_OK(GetFlightInfoImportedKeys(command, context, request, info)); } - return Status::OK(); + return Status::Invalid("The defined request is invalid."); } Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket& request, @@ -123,7 +123,7 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket ARROW_RETURN_NOT_OK(DoGetImportedKeys(command, context, stream)); } - return Status::OK(); + return Status::Invalid("The defined request is invalid."); } Status FlightSqlServerBase::GetFlightInfoCatalogs(const ServerCallContext& context, From 87fc8aae9cb0e5819dadc2f3a1ba394904f2acc2 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 24 Sep 2021 15:38:02 -0300 Subject: [PATCH 026/237] Implement Flight SQL example server using SQLite3 --- .../arrow/flight/flight-sql/CMakeLists.txt | 11 ++ .../flight-sql/example/sqlite_server.cc | 117 +++++++++++++ .../flight/flight-sql/example/sqlite_server.h | 56 +++++++ .../flight-sql/example/sqlite_statement.cc | 76 +++++++++ .../flight-sql/example/sqlite_statement.h | 48 ++++++ .../example/sqlite_statement_batch_reader.cc | 156 ++++++++++++++++++ .../example/sqlite_statement_batch_reader.h | 51 ++++++ .../arrow/flight/flight-sql/test_server.cc | 55 ++++++ 8 files changed, 570 insertions(+) create mode 100644 cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc create mode 100644 cpp/src/arrow/flight/flight-sql/example/sqlite_server.h create mode 100644 cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc create mode 100644 cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h create mode 100644 cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc create mode 100644 cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h create mode 100644 cpp/src/arrow/flight/flight-sql/test_server.cc diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 5cc000171ac..16316640eee 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -95,3 +95,14 @@ add_arrow_test(flight_sql_test add_executable(flight_sql_test_app test_app.cc) target_link_libraries(flight_sql_test_app PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES}) + +find_package(SQLite3) +include_directories(${SQLite3_INCLUDE_DIRS}) + +add_executable(flight_sql_test_server + sql_server.cpp + test_server.cc example/sqlite_statement.cc + example/sqlite_statement_batch_reader.cc example/sqlite_server.cc) +target_link_libraries(flight_sql_test_server + PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES} + ${SQLite3_LIBRARIES}) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc new file mode 100644 index 00000000000..70c30de560c --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -0,0 +1,117 @@ +// 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 "arrow/api.h" +#include "arrow/flight/flight-sql/sql_server.h" +#include "arrow/flight/flight-sql/example/sqlite_server.h" +#include "arrow/flight/flight-sql/example/sqlite_statement.h" +#include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +SQLiteFlightSqlServer::SQLiteFlightSqlServer() { + db_ = NULLPTR; + if (sqlite3_open(":memory:", &db_)) { + sqlite3_close(db_); + throw std::runtime_error(std::string("Can't open database: ") + sqlite3_errmsg(db_)); + } + + ExecuteSql( + "CREATE TABLE COMPANY(" + "ID INT PRIMARY KEY NOT NULL," + "NAME TEXT NOT NULL," + "AGE INT NOT NULL," + "ADDRESS CHAR(50)," + "SALARY REAL );"); + + ExecuteSql( + "INSERT INTO COMPANY (ID,NAME,AGE,ADDRESS,SALARY) " + "VALUES (1, 'Paul', 32, 'California', 20000.00 ); " + "INSERT INTO COMPANY (ID,NAME,AGE,ADDRESS,SALARY)" + "VALUES (2, 'Allen', 25, 'Texas', 15000.00 ); " + "INSERT INTO COMPANY (ID,NAME,AGE,ADDRESS,SALARY)" + "VALUES (3, 'Teddy', 23, 'Norway', 20000.00 );" + "INSERT INTO COMPANY (ID,NAME,AGE,ADDRESS,SALARY)" + "VALUES (4, 'Mark', 25, 'Rich', 65000.00 );"); +} + +SQLiteFlightSqlServer::~SQLiteFlightSqlServer() { sqlite3_close(db_); } + +Status SQLiteFlightSqlServer::GetFlightInfoStatement( + const pb::sql::CommandStatementQuery& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + const std::string &query = command.query(); + + std::shared_ptr statement; + ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, query, &statement)); + + std::shared_ptr schema; + ARROW_RETURN_NOT_OK(statement->GetSchema(&schema)); + + pb::sql::TicketStatementQuery ticket_statement_query; + ticket_statement_query.set_statement_handle(query); + + google::protobuf::Any ticket; + ticket.PackFrom(ticket_statement_query); + + const std::string &ticket_string = ticket.SerializeAsString(); + std::vector endpoints{FlightEndpoint{{ticket_string}, {}}}; + ARROW_ASSIGN_OR_RAISE(auto result, + FlightInfo::Make(*schema, descriptor, endpoints, -1, -1)) + + *info = std::unique_ptr(new FlightInfo(result)); + + return Status::OK(); +} + +Status SQLiteFlightSqlServer::DoGetStatement(const pb::sql::TicketStatementQuery &command, + const ServerCallContext &context, + std::unique_ptr *result) { + const std::string &string = command.statement_handle(); + std::shared_ptr batch_reader; + + std::shared_ptr statement; + ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, string, &statement)); + + std::shared_ptr reader; + ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Make(statement, &reader)); + + *result = std::unique_ptr(new RecordBatchStream(reader)); + + return Status::OK(); +} + +void SQLiteFlightSqlServer::ExecuteSql(const std::string &sql) { + char *zErrMsg = NULLPTR; + int rc = sqlite3_exec(db_, sql.data(), NULLPTR, NULLPTR, &zErrMsg); + if (rc != SQLITE_OK) { + fprintf(stderr, "SQL error: %s\n", zErrMsg); + sqlite3_free(zErrMsg); + } else { + fprintf(stdout, "Executed successfully\n"); + } +} + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h new file mode 100644 index 00000000000..4d97e46b9b7 --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -0,0 +1,56 @@ +// 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 "arrow/api.h" +#include "arrow/flight/flight-sql/sql_server.h" +#include "arrow/flight/flight-sql/example/sqlite_statement.h" +#include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +class SQLiteFlightSqlServer : public FlightSqlServerBase { + public: + SQLiteFlightSqlServer(); + + ~SQLiteFlightSqlServer() override; + + Status GetFlightInfoStatement(const pb::sql::CommandStatementQuery &command, + const ServerCallContext &context, + const FlightDescriptor &descriptor, + std::unique_ptr *info) override; + + Status DoGetStatement(const pb::sql::TicketStatementQuery &command, + const ServerCallContext &context, + std::unique_ptr *result) override; + + private: + sqlite3 *db_; + + void ExecuteSql(const std::string &sql); +}; + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc new file mode 100644 index 00000000000..97e330c83c3 --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -0,0 +1,76 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include + +#include "arrow/api.h" +#include "arrow/flight/flight-sql/example/sqlite_statement.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(); + } +} + +Status SqliteStatement::Create(sqlite3 *db, const std::string &sql, + std::shared_ptr *result) { + sqlite3_stmt *stmt; + int rc = + sqlite3_prepare_v2(db, sql.c_str(), static_cast(sql.length()), &stmt, NULLPTR); + + if (rc != SQLITE_OK) { + sqlite3_finalize(stmt); + return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db)); + } + + result->reset(new SqliteStatement(db, stmt)); + return Status::OK(); +} + +Status SqliteStatement::GetSchema(std::shared_ptr *schema) 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); + const int column_type = sqlite3_column_type(stmt, i); + + std::shared_ptr data_type = GetDataTypeFromSqliteType(column_type); + fields.push_back(arrow::field(column_name, data_type)); + } + + *schema = arrow::schema(fields); + return Status::OK(); +} + +SqliteStatement::~SqliteStatement() { + sqlite3_finalize(stmt); +} + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h new file mode 100644 index 00000000000..b631d117efc --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h @@ -0,0 +1,48 @@ +// 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 "arrow/api.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +class SqliteStatement { + public: + static Status Create(sqlite3 *db, const std::string &sql, + std::shared_ptr *result); + + ~SqliteStatement(); + + Status GetSchema(std::shared_ptr *schema) const; + + sqlite3_stmt *stmt; + sqlite3 *db; + + private: + SqliteStatement(sqlite3 *_db, sqlite3_stmt *_stmt) : stmt(_stmt), db(_db) {} +}; + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc new file mode 100644 index 00000000000..552f3243d9c --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -0,0 +1,156 @@ +// 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/flight-sql/example/sqlite_statement.h" +#include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" + +#include + +#include "arrow/api.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); \ + ARROW_RETURN_NOT_OK( \ + (dynamic_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); \ + ARROW_RETURN_NOT_OK( \ + (dynamic_cast(builder)).Append((char*)blob, bytes)); \ + break; \ + } + +#define INT_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ + case TYPE_CLASS##Type::type_id: { \ + sqlite3_int64 value = sqlite3_column_int64(STMT, COLUMN); \ + ARROW_RETURN_NOT_OK((dynamic_cast(builder)).Append(value)); \ + break; \ + } + +#define FLOAT_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ + case TYPE_CLASS##Type::type_id: { \ + double value = sqlite3_column_double(STMT, COLUMN); \ + ARROW_RETURN_NOT_OK((dynamic_cast(builder)).Append(value)); \ + break; \ + } + +#define MAX_BATCH_SIZE 1024 + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +std::shared_ptr SqliteStatementBatchReader::schema() const { return schema_; } + +SqliteStatementBatchReader::SqliteStatementBatchReader( + std::shared_ptr statement, std::shared_ptr schema, int rc) + : statement_(std::move(statement)), schema_(std::move(schema)), rc_(rc) {} + +Status SqliteStatementBatchReader::Make( + const std::shared_ptr &statement_, + std::shared_ptr *result) { + sqlite3_stmt *stmt_ = statement_->stmt; + sqlite3 *db_ = statement_->db; + + int rc = sqlite3_step(stmt_); + if (rc == SQLITE_ERROR) { + return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db_)); + } + + std::shared_ptr schema; + ARROW_RETURN_NOT_OK(statement_->GetSchema(&schema)); + + result->reset(new SqliteStatementBatchReader(statement_, schema, rc)); + + return Status::OK(); +} + +Status SqliteStatementBatchReader::ReadNext(std::shared_ptr *out) { + sqlite3_stmt *stmt_ = statement_->stmt; + sqlite3 *db_ = statement_->db; + + 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])); + } + + int rows = 0; + while (rows < MAX_BATCH_SIZE && 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]; + + 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()); + } + } + + rc_ = sqlite3_step(stmt_); + if (rc_ == SQLITE_ERROR) { + return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db_)); + } + } + + 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/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h new file mode 100644 index 00000000000..6e887f16219 --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h @@ -0,0 +1,51 @@ +// 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 "arrow/api.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +class SqliteStatementBatchReader : public RecordBatchReader { + public: + static Status Make(const std::shared_ptr &statement_, + std::shared_ptr *result); + + std::shared_ptr schema() const override; + + Status ReadNext(std::shared_ptr *out) override; + + private: + std::shared_ptr statement_; + std::shared_ptr schema_; + int rc_; + + SqliteStatementBatchReader(std::shared_ptr statement, + std::shared_ptr schema, + int rc); +}; + +} // namespace example +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/test_server.cc b/cpp/src/arrow/flight/flight-sql/test_server.cc new file mode 100644 index 00000000000..c0f8725f757 --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/test_server.cc @@ -0,0 +1,55 @@ +// 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/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" +#include "arrow/flight/flight-sql/example/sqlite_server.h" + +DEFINE_int32(port, 31337, "Server port to listen on"); + +namespace pb = arrow::flight::protocol; + +std::unique_ptr g_server; + +int main(int argc, char **argv) { + gflags::SetUsageMessage("Integration testing server for Flight SQL."); + gflags::ParseCommandLineFlags(&argc, &argv, true); + + arrow::flight::Location location; + ARROW_CHECK_OK(arrow::flight::Location::ForGrpcTcp("0.0.0.0", FLAGS_port, &location)); + arrow::flight::FlightServerOptions options(location); + + g_server.reset(new arrow::flight::sql::example::SQLiteFlightSqlServer()); + + ARROW_CHECK_OK(g_server->Init(options)); + // Exit with a clean error code (0) on SIGTERM + ARROW_CHECK_OK(g_server->SetShutdownOnSignals({SIGTERM})); + + std::cout << "Server listening on localhost:" << g_server->port() << std::endl; + ARROW_CHECK_OK(g_server->Serve()); + return 0; +} From 3584fc07f7fd76ab46ab76215d609cb35d344c91 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 24 Sep 2021 15:42:35 -0300 Subject: [PATCH 027/237] Add missing sqlite3 dependency on vcpkg.json --- cpp/vcpkg.json | 1 + 1 file changed, 1 insertion(+) 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", From f7b6461ee156d05e602f4af287d1ae7be54794f2 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 24 Sep 2021 16:04:58 -0300 Subject: [PATCH 028/237] Add documentation to example classes --- .../flight-sql/example/sqlite_server.cc | 22 ++++++++---------- .../flight/flight-sql/example/sqlite_server.h | 8 +++++-- .../flight-sql/example/sqlite_statement.cc | 21 +++++++++++++---- .../flight-sql/example/sqlite_statement.h | 23 ++++++++++++++++--- .../example/sqlite_statement_batch_reader.cc | 18 ++++----------- .../example/sqlite_statement_batch_reader.h | 9 ++++++-- 6 files changed, 65 insertions(+), 36 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 70c30de560c..d86f526f2f6 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -56,6 +56,15 @@ SQLiteFlightSqlServer::SQLiteFlightSqlServer() { SQLiteFlightSqlServer::~SQLiteFlightSqlServer() { sqlite3_close(db_); } +void SQLiteFlightSqlServer::ExecuteSql(const std::string &sql) { + char *zErrMsg = NULLPTR; + int rc = sqlite3_exec(db_, sql.data(), NULLPTR, NULLPTR, &zErrMsg); + if (rc != SQLITE_OK) { + fprintf(stderr, "SQL error: %s\n", zErrMsg); + sqlite3_free(zErrMsg); + } +} + Status SQLiteFlightSqlServer::GetFlightInfoStatement( const pb::sql::CommandStatementQuery& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { @@ -93,24 +102,13 @@ Status SQLiteFlightSqlServer::DoGetStatement(const pb::sql::TicketStatementQuery ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, string, &statement)); std::shared_ptr reader; - ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Make(statement, &reader)); + ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create(statement, &reader)); *result = std::unique_ptr(new RecordBatchStream(reader)); return Status::OK(); } -void SQLiteFlightSqlServer::ExecuteSql(const std::string &sql) { - char *zErrMsg = NULLPTR; - int rc = sqlite3_exec(db_, sql.data(), NULLPTR, NULLPTR, &zErrMsg); - if (rc != SQLITE_OK) { - fprintf(stderr, "SQL error: %s\n", zErrMsg); - sqlite3_free(zErrMsg); - } else { - fprintf(stdout, "Executed successfully\n"); - } -} - } // namespace example } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index 4d97e46b9b7..36642ad01f2 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -29,12 +29,18 @@ namespace flight { namespace sql { namespace example { +/// \brief Example implementation of FlightSqlServerBase backed by an in-memory SQLite3 +/// database. class SQLiteFlightSqlServer : public FlightSqlServerBase { public: SQLiteFlightSqlServer(); ~SQLiteFlightSqlServer() override; + /// \brief Auxiliary method used to execute an arbitrary SQL statement on the underlying + /// SQLite database. + void ExecuteSql(const std::string &sql); + Status GetFlightInfoStatement(const pb::sql::CommandStatementQuery &command, const ServerCallContext &context, const FlightDescriptor &descriptor, @@ -46,8 +52,6 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { private: sqlite3 *db_; - - void ExecuteSql(const std::string &sql); }; } // namespace example diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index 97e330c83c3..66a25b2ec31 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -53,10 +53,10 @@ Status SqliteStatement::Create(sqlite3 *db, const std::string &sql, Status SqliteStatement::GetSchema(std::shared_ptr *schema) const { std::vector> fields; - int column_count = sqlite3_column_count(stmt); + int column_count = sqlite3_column_count(stmt_); for (int i = 0; i < column_count; i++) { - const char *column_name = sqlite3_column_name(stmt, i); - const int column_type = sqlite3_column_type(stmt, i); + const char *column_name = sqlite3_column_name(stmt_, i); + const int column_type = sqlite3_column_type(stmt_, i); std::shared_ptr data_type = GetDataTypeFromSqliteType(column_type); fields.push_back(arrow::field(column_name, data_type)); @@ -67,7 +67,20 @@ Status SqliteStatement::GetSchema(std::shared_ptr *schema) const { } SqliteStatement::~SqliteStatement() { - sqlite3_finalize(stmt); + sqlite3_finalize(stmt_); +} + +Status SqliteStatement::Step(int *rc) { + *rc = sqlite3_step(stmt_); + if (*rc == SQLITE_ERROR) { + return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db_)); + } + + return Status::OK(); +} + +sqlite3_stmt *SqliteStatement::GetSqlite3Stmt() { + return stmt_; } } // namespace example diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h index b631d117efc..06841b27878 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h @@ -28,18 +28,35 @@ namespace example { class SqliteStatement { public: + + /// \brief Creates a SQLite3 statement. + /// \param[in] db SQLite3 database instance. + /// \param[in] sql SQL statement. + /// \param[out] result The resulting SqliteStatement. + /// \return Status. static Status Create(sqlite3 *db, const std::string &sql, std::shared_ptr *result); ~SqliteStatement(); + /// \brief Creates an Arrow Schema based on the results of this statement. + /// \param[out] schema The resulting Schema. + /// \return Status. Status GetSchema(std::shared_ptr *schema) const; - sqlite3_stmt *stmt; - sqlite3 *db; + /// \brief Steps on underlying sqlite3_stmt. + /// \param[out] rc The resulting return code from SQLite. + /// \return Status. + Status Step(int* rc); + + /// \brief Returns the underlying sqlite3_stmt. + sqlite3_stmt *GetSqlite3Stmt(); private: - SqliteStatement(sqlite3 *_db, sqlite3_stmt *_stmt) : stmt(_stmt), db(_db) {} + sqlite3 *db_; + sqlite3_stmt *stmt_; + + SqliteStatement(sqlite3 *db, sqlite3_stmt *stmt) : db_(db), stmt_(stmt) {} }; } // namespace example diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index 552f3243d9c..2d17373a9e7 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -67,16 +67,12 @@ SqliteStatementBatchReader::SqliteStatementBatchReader( std::shared_ptr statement, std::shared_ptr schema, int rc) : statement_(std::move(statement)), schema_(std::move(schema)), rc_(rc) {} -Status SqliteStatementBatchReader::Make( +Status SqliteStatementBatchReader::Create( const std::shared_ptr &statement_, std::shared_ptr *result) { - sqlite3_stmt *stmt_ = statement_->stmt; - sqlite3 *db_ = statement_->db; - int rc = sqlite3_step(stmt_); - if (rc == SQLITE_ERROR) { - return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db_)); - } + int rc; + ARROW_RETURN_NOT_OK(statement_->Step(&rc)); std::shared_ptr schema; ARROW_RETURN_NOT_OK(statement_->GetSchema(&schema)); @@ -87,8 +83,7 @@ Status SqliteStatementBatchReader::Make( } Status SqliteStatementBatchReader::ReadNext(std::shared_ptr *out) { - sqlite3_stmt *stmt_ = statement_->stmt; - sqlite3 *db_ = statement_->db; + sqlite3_stmt *stmt_ = statement_->GetSqlite3Stmt(); const int num_fields = schema_->num_fields(); std::vector> builders(num_fields); @@ -130,10 +125,7 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr *out) { } } - rc_ = sqlite3_step(stmt_); - if (rc_ == SQLITE_ERROR) { - return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db_)); - } + ARROW_RETURN_NOT_OK(statement_->Step(&rc_)); } if (rows > 0) { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h index 6e887f16219..a4941a9f64a 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h @@ -28,8 +28,13 @@ namespace example { class SqliteStatementBatchReader : public RecordBatchReader { public: - static Status Make(const std::shared_ptr &statement_, - std::shared_ptr *result); + + /// \brief Creates a RecordBatchReader backed by a SQLite statement. + /// \param[in] statement SQLite statement to be read. + /// \param[out] result The resulting RecordBatchReader. + /// \return Status. + static Status Create(const std::shared_ptr &statement, + std::shared_ptr *result); std::shared_ptr schema() const override; From 2232dd67f3829aa61feea031809dad4543e75c21 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 27 Sep 2021 16:36:06 -0300 Subject: [PATCH 029/237] Add integration tests for Flight SQL server example --- .../arrow/flight/flight-sql/CMakeLists.txt | 8 +- cpp/src/arrow/flight/flight-sql/client_impl.h | 2 +- .../arrow/flight/flight-sql/client_test.cc | 27 ++-- .../flight-sql/example/sqlite_server.cc | 40 ++++-- .../flight/flight-sql/sql_server_test.cc | 117 ++++++++++++++++++ 5 files changed, 166 insertions(+), 28 deletions(-) create mode 100644 cpp/src/arrow/flight/flight-sql/sql_server_test.cc diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 16316640eee..30051358ec1 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -51,7 +51,7 @@ set(CMAKE_CXX_FLAGS_BACKUP "${CMAKE_CXX_FLAGS}") string(REPLACE "/WX" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") string(REPLACE "-Werror " " " CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") -set(ARROW_FLIGHT_SQL_SRCS protocol_internal.cc) +set(ARROW_FLIGHT_SQL_SRCS protocol_internal.cc sql_server.cpp) add_arrow_lib(arrow_flight_sql CMAKE_PACKAGE_NAME @@ -77,15 +77,17 @@ add_arrow_lib(arrow_flight_sql if(ARROW_TEST_LINKAGE STREQUAL "static") set(ARROW_FLIGHT_SQL_TEST_LINK_LIBS - arrow_flight_sql_static ${ARROW_FLIGHT_STATIC_LINK_LIBS} ${ARROW_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_TEST_LINK_LIBS}) + set(ARROW_FLIGHT_SQL_TEST_LINK_LIBS arrow_flight_sql_shared arrow_flight_testing_shared ${ARROW_TEST_LINK_LIBS}) endif() add_arrow_test(flight_sql_test SOURCES client_test.cc + sql_server_test.cc STATIC_LINK_LIBS + ${ARROW_FLIGHT_SQL_LINK_LIBS} ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} GTest::gtest GTest::gmock diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 3abf9ba02b6..db1e2187d3c 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -36,7 +36,7 @@ FlightSqlClientT::FlightSqlClientT(std::unique_ptr& client) { template FlightSqlClientT::~FlightSqlClientT() = default; -FlightDescriptor GetFlightDescriptorForCommand(const google::protobuf::Message& command) { +inline FlightDescriptor GetFlightDescriptorForCommand(const google::protobuf::Message& command) { google::protobuf::Any any; any.PackFrom(command); diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 66a3251c2da..1f5173b9a36 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -27,7 +27,6 @@ namespace pb = arrow::flight::protocol; using ::testing::_; using ::testing::Ref; - namespace arrow { namespace flight { namespace sql { @@ -38,12 +37,12 @@ class FlightClientMock { public: MOCK_METHOD(Status, GetFlightInfo, (const FlightCallOptions&, const FlightDescriptor&, - std::unique_ptr*)); + std::unique_ptr*)); MOCK_METHOD(Status, DoPut, (const FlightCallOptions&, const FlightDescriptor&, - const std::shared_ptr& schema, - std::unique_ptr*, - std::unique_ptr*)); + const std::shared_ptr& schema, + std::unique_ptr*, + std::unique_ptr*)); }; class FlightMetadataReaderMock : public FlightMetadataReader { @@ -68,7 +67,7 @@ FlightDescriptor getDescriptor(google::protobuf::Message& command) { return FlightDescriptor::Command(string); } -TEST(TestFlightSql, TestGetCatalogs) { +TEST(TestFlightSqlClient, TestGetCatalogs) { auto *client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); @@ -84,7 +83,7 @@ TEST(TestFlightSql, TestGetCatalogs) { (void) sqlClient.GetCatalogs(call_options, &flight_info); } -TEST(TestFlightSql, TestGetSchemas) { +TEST(TestFlightSqlClient, TestGetSchemas) { auto *client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); @@ -106,7 +105,7 @@ TEST(TestFlightSql, TestGetSchemas) { &schema_filter_pattern, &flight_info); } -TEST(TestFlightSql, TestGetTables) { +TEST(TestFlightSqlClient, TestGetTables) { auto *client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); @@ -137,7 +136,7 @@ TEST(TestFlightSql, TestGetTables) { table_types, &flight_info); } -TEST(TestFlightSql, TestGetTableTypes) { +TEST(TestFlightSqlClient, TestGetTableTypes) { auto *client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); @@ -153,7 +152,7 @@ TEST(TestFlightSql, TestGetTableTypes) { (void) sqlClient.GetTableTypes(call_options, &flight_info); } -TEST(TestFlightSql, TestGetExported) { +TEST(TestFlightSqlClient, TestGetExported) { auto *client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); @@ -176,7 +175,7 @@ TEST(TestFlightSql, TestGetExported) { (void) sqlClient.GetExportedKeys(call_options, &catalog, &schema, table, &flight_info); } -TEST(TestFlightSql, TestGetImported) { +TEST(TestFlightSqlClient, TestGetImported) { auto *client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); @@ -199,7 +198,7 @@ TEST(TestFlightSql, TestGetImported) { (void) sqlClient.GetImportedKeys(call_options, &catalog, &schema, table, &flight_info); } -TEST(TestFlightSql, TestGetPrimary) { +TEST(TestFlightSqlClient, TestGetPrimary) { auto *client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); @@ -222,7 +221,7 @@ TEST(TestFlightSql, TestGetPrimary) { (void) sqlClient.GetPrimaryKeys(call_options, &catalog, &schema, table, &flight_info); } -TEST(TestFlightSql, TestExecute) { +TEST(TestFlightSqlClient, TestExecute) { auto *client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); @@ -241,7 +240,7 @@ TEST(TestFlightSql, TestExecute) { (void) sqlClient.Execute(call_options, query, &flight_info); } -TEST(TestFlightSql, TestExecuteUpdate) { +TEST(TestFlightSqlClient, TestExecuteUpdate) { auto *client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index d86f526f2f6..9d703d003ea 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -15,13 +15,14 @@ // specific language governing permissions and limitations // under the License. +#include "arrow/flight/flight-sql/example/sqlite_server.h" + #include #include "arrow/api.h" -#include "arrow/flight/flight-sql/sql_server.h" -#include "arrow/flight/flight-sql/example/sqlite_server.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/flight-sql/sql_server.h" namespace arrow { namespace flight { @@ -35,6 +36,25 @@ SQLiteFlightSqlServer::SQLiteFlightSqlServer() { throw std::runtime_error(std::string("Can't open database: ") + sqlite3_errmsg(db_)); } + ExecuteSql( + "CREATE TABLE foreignTable (" + "id INTEGER PRIMARY KEY AUTOINCREMENT, " + "foreignName varchar(100), " + "value int)"); + ExecuteSql( + "CREATE TABLE intTable (" + "id INTEGER PRIMARY KEY AUTOINCREMENT, " + "keyName varchar(100), " + "value int, " + "foreignId int references foreignTable(id))"); + ExecuteSql("INSERT INTO foreignTable (foreignName, value) VALUES ('keyOne', 1)"); + ExecuteSql("INSERT INTO foreignTable (foreignName, value) VALUES ('keyTwo', 0)"); + ExecuteSql("INSERT INTO foreignTable (foreignName, value) VALUES ('keyThree', -1)"); + ExecuteSql("INSERT INTO intTable (keyName, value, foreignId) VALUES ('one', 1, 1)"); + ExecuteSql("INSERT INTO intTable (keyName, value, foreignId) VALUES ('zero', 0, 1)"); + ExecuteSql( + "INSERT INTO intTable (keyName, value, foreignId) VALUES ('negative one', -1, 1)"); + ExecuteSql( "CREATE TABLE COMPANY(" "ID INT PRIMARY KEY NOT NULL," @@ -56,8 +76,8 @@ SQLiteFlightSqlServer::SQLiteFlightSqlServer() { SQLiteFlightSqlServer::~SQLiteFlightSqlServer() { sqlite3_close(db_); } -void SQLiteFlightSqlServer::ExecuteSql(const std::string &sql) { - char *zErrMsg = NULLPTR; +void SQLiteFlightSqlServer::ExecuteSql(const std::string& sql) { + char* zErrMsg = NULLPTR; int rc = sqlite3_exec(db_, sql.data(), NULLPTR, NULLPTR, &zErrMsg); if (rc != SQLITE_OK) { fprintf(stderr, "SQL error: %s\n", zErrMsg); @@ -68,7 +88,7 @@ void SQLiteFlightSqlServer::ExecuteSql(const std::string &sql) { Status SQLiteFlightSqlServer::GetFlightInfoStatement( const pb::sql::CommandStatementQuery& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - const std::string &query = command.query(); + const std::string& query = command.query(); std::shared_ptr statement; ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, query, &statement)); @@ -82,7 +102,7 @@ Status SQLiteFlightSqlServer::GetFlightInfoStatement( google::protobuf::Any ticket; ticket.PackFrom(ticket_statement_query); - const std::string &ticket_string = ticket.SerializeAsString(); + const std::string& ticket_string = ticket.SerializeAsString(); std::vector endpoints{FlightEndpoint{{ticket_string}, {}}}; ARROW_ASSIGN_OR_RAISE(auto result, FlightInfo::Make(*schema, descriptor, endpoints, -1, -1)) @@ -92,10 +112,10 @@ Status SQLiteFlightSqlServer::GetFlightInfoStatement( return Status::OK(); } -Status SQLiteFlightSqlServer::DoGetStatement(const pb::sql::TicketStatementQuery &command, - const ServerCallContext &context, - std::unique_ptr *result) { - const std::string &string = command.statement_handle(); +Status SQLiteFlightSqlServer::DoGetStatement(const pb::sql::TicketStatementQuery& command, + const ServerCallContext& context, + std::unique_ptr* result) { + const std::string& string = command.statement_handle(); std::shared_ptr batch_reader; std::shared_ptr statement; diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc new file mode 100644 index 00000000000..e6fc3742aa7 --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -0,0 +1,117 @@ +// 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 +#include +#include +#include +#include + +#define unparen(...) __VA_ARGS__ +#define DECLARE_ARRAY(ARRAY_NAME, TYPE_CLASS, DATA) \ + std::shared_ptr ARRAY_NAME; \ + { \ + arrow::TYPE_CLASS##Builder builder; \ + auto data = unparen DATA; \ + for (const auto& item : data) { \ + ASSERT_OK(builder.Append(item)); \ + } \ + ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ + } + +using ::testing::_; +using ::testing::Ref; + +namespace pb = arrow::flight::protocol; + +namespace arrow { +namespace flight { +namespace sql { + +void ASSERT_OK(Status x) {} + +/* + * FIXME + * For some reason when the first protobuf message used on "any.PackFrom()" has a + * string field (like CommandStatementQuery), libprotobuf throws: + * + * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3624] Invalid proto descriptor for file "google/protobuf/descriptor.proto": + * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3627] google/protobuf/descriptor.proto: Unrecognized syntax: SELECT * XXXXXX + * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3624] Invalid proto descriptor for file "FlightSql.proto": + * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3627] arrow.flight.protocol.sql.experimental: ".google.protobuf.MessageOptions" is not defined. + * [libprotobuf FATAL .../src/google/protobuf/generated_message_reflection.cc:2457] CHECK failed: file != nullptr: + * + * If the first protobuf message used on "any.PackFrom()" has no fields, all the + * subsequent calls misteriously work. + * + * Still don't know how to fix this, so we are keeping this dummy tests at the beginning + * of the test suite to move forward. + */ +TEST(TestFlightSqlServer, FIX_PROTOBUF_BUG) { + pb::sql::CommandGetCatalogs command; + google::protobuf::Any any; + any.PackFrom(command); +} + +TEST(TestFlightSqlServer, TestCommandStatementQuery) { + TestServer server("flight_sql_test_server"); + server.Start(); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + ASSERT_TRUE(server.IsRunning()); + + std::stringstream ss; + ss << "grpc://localhost:" << server.port(); + std::string uri = ss.str(); + + std::unique_ptr client; + Location location; + ASSERT_OK(Location::Parse(uri, &location)); + ASSERT_OK(FlightClient::Connect(location, &client)); + + FlightSqlClient sql_client(client); + + std::unique_ptr flight_info; + ASSERT_OK(sql_client.Execute({}, "SELECT * FROM intTable", &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + 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())}); + + DECLARE_ARRAY(id_array, Int64, ({1, 2, 3})); + DECLARE_ARRAY(keyname_array, String, ({"one", "zero", "negative one"})); + DECLARE_ARRAY(value_array, Int64, ({1, 0, -1})); + DECLARE_ARRAY(foreignId_array, Int64, ({1, 1, 1})); + + const std::shared_ptr
& expected_table = Table::Make( + expected_schema, {id_array, keyname_array, value_array, foreignId_array}); + + ASSERT_TRUE(expected_table->Equals(*table)); +} + +} // namespace sql +} // namespace flight +} // namespace arrow From 93c5a4ba70e2dd6b2083c8235255af095a3a703b Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 28 Sep 2021 14:27:29 -0300 Subject: [PATCH 030/237] Fix checkstyle errors --- cpp/src/arrow/flight/flight-sql/CMakeLists.txt | 12 ++++++++---- cpp/src/arrow/flight/flight-sql/client_impl.h | 3 ++- .../flight/flight-sql/example/sqlite_statement.h | 1 - .../example/sqlite_statement_batch_reader.h | 1 - 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 30051358ec1..3978a6a3b07 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -77,9 +77,11 @@ add_arrow_lib(arrow_flight_sql if(ARROW_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}) + 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}) + set(ARROW_FLIGHT_SQL_TEST_LINK_LIBS arrow_flight_sql_shared arrow_flight_testing_shared + ${ARROW_TEST_LINK_LIBS}) endif() add_arrow_test(flight_sql_test @@ -103,8 +105,10 @@ include_directories(${SQLite3_INCLUDE_DIRS}) add_executable(flight_sql_test_server sql_server.cpp - test_server.cc example/sqlite_statement.cc - example/sqlite_statement_batch_reader.cc example/sqlite_server.cc) + test_server.cc + example/sqlite_statement.cc + example/sqlite_statement_batch_reader.cc + example/sqlite_server.cc) target_link_libraries(flight_sql_test_server PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES} ${SQLite3_LIBRARIES}) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index db1e2187d3c..c1fbb40f43e 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -36,7 +36,8 @@ FlightSqlClientT::FlightSqlClientT(std::unique_ptr& client) { template FlightSqlClientT::~FlightSqlClientT() = default; -inline FlightDescriptor GetFlightDescriptorForCommand(const google::protobuf::Message& command) { +inline FlightDescriptor GetFlightDescriptorForCommand( + const google::protobuf::Message& command) { google::protobuf::Any any; any.PackFrom(command); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h index 06841b27878..a0108dbffe6 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h @@ -28,7 +28,6 @@ namespace example { class SqliteStatement { public: - /// \brief Creates a SQLite3 statement. /// \param[in] db SQLite3 database instance. /// \param[in] sql SQL statement. diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h index a4941a9f64a..2aa80a6086f 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h @@ -28,7 +28,6 @@ namespace example { class SqliteStatementBatchReader : public RecordBatchReader { public: - /// \brief Creates a RecordBatchReader backed by a SQLite statement. /// \param[in] statement SQLite statement to be read. /// \param[out] result The resulting RecordBatchReader. From 68bec018bd9784327d27dc80b80c84e18077606c Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 28 Sep 2021 16:07:07 -0300 Subject: [PATCH 031/237] Fix minor comments on PR --- cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 9d703d003ea..b0970bbd52c 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -78,7 +78,7 @@ SQLiteFlightSqlServer::~SQLiteFlightSqlServer() { sqlite3_close(db_); } void SQLiteFlightSqlServer::ExecuteSql(const std::string& sql) { char* zErrMsg = NULLPTR; - int rc = sqlite3_exec(db_, sql.data(), NULLPTR, NULLPTR, &zErrMsg); + int rc = sqlite3_exec(db_, sql.c_str(), NULLPTR, NULLPTR, &zErrMsg); if (rc != SQLITE_OK) { fprintf(stderr, "SQL error: %s\n", zErrMsg); sqlite3_free(zErrMsg); @@ -115,11 +115,10 @@ Status SQLiteFlightSqlServer::GetFlightInfoStatement( Status SQLiteFlightSqlServer::DoGetStatement(const pb::sql::TicketStatementQuery& command, const ServerCallContext& context, std::unique_ptr* result) { - const std::string& string = command.statement_handle(); - std::shared_ptr batch_reader; + const std::string& sql = command.statement_handle(); std::shared_ptr statement; - ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, string, &statement)); + ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, sql, &statement)); std::shared_ptr reader; ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create(statement, &reader)); From 5d3bc66c200abc30fd421ac7eb0c1c6bb87985cb Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 28 Sep 2021 16:30:11 -0300 Subject: [PATCH 032/237] Improve readability for SqliteFlightSqlServer setup --- .../flight-sql/example/sqlite_server.cc | 55 +++++++------------ 1 file changed, 19 insertions(+), 36 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index b0970bbd52c..cf7497bfb29 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -36,42 +36,25 @@ SQLiteFlightSqlServer::SQLiteFlightSqlServer() { throw std::runtime_error(std::string("Can't open database: ") + sqlite3_errmsg(db_)); } - ExecuteSql( - "CREATE TABLE foreignTable (" - "id INTEGER PRIMARY KEY AUTOINCREMENT, " - "foreignName varchar(100), " - "value int)"); - ExecuteSql( - "CREATE TABLE intTable (" - "id INTEGER PRIMARY KEY AUTOINCREMENT, " - "keyName varchar(100), " - "value int, " - "foreignId int references foreignTable(id))"); - ExecuteSql("INSERT INTO foreignTable (foreignName, value) VALUES ('keyOne', 1)"); - ExecuteSql("INSERT INTO foreignTable (foreignName, value) VALUES ('keyTwo', 0)"); - ExecuteSql("INSERT INTO foreignTable (foreignName, value) VALUES ('keyThree', -1)"); - ExecuteSql("INSERT INTO intTable (keyName, value, foreignId) VALUES ('one', 1, 1)"); - ExecuteSql("INSERT INTO intTable (keyName, value, foreignId) VALUES ('zero', 0, 1)"); - ExecuteSql( - "INSERT INTO intTable (keyName, value, foreignId) VALUES ('negative one', -1, 1)"); - - ExecuteSql( - "CREATE TABLE COMPANY(" - "ID INT PRIMARY KEY NOT NULL," - "NAME TEXT NOT NULL," - "AGE INT NOT NULL," - "ADDRESS CHAR(50)," - "SALARY REAL );"); - - ExecuteSql( - "INSERT INTO COMPANY (ID,NAME,AGE,ADDRESS,SALARY) " - "VALUES (1, 'Paul', 32, 'California', 20000.00 ); " - "INSERT INTO COMPANY (ID,NAME,AGE,ADDRESS,SALARY)" - "VALUES (2, 'Allen', 25, 'Texas', 15000.00 ); " - "INSERT INTO COMPANY (ID,NAME,AGE,ADDRESS,SALARY)" - "VALUES (3, 'Teddy', 23, 'Norway', 20000.00 );" - "INSERT INTO COMPANY (ID,NAME,AGE,ADDRESS,SALARY)" - "VALUES (4, 'Mark', 25, 'Rich', 65000.00 );"); + 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); +)"); } SQLiteFlightSqlServer::~SQLiteFlightSqlServer() { sqlite3_close(db_); } From 7add18f4f40f7865298c31f7a3a523be2e191f9e Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 28 Sep 2021 15:07:40 -0300 Subject: [PATCH 033/237] Implement CommandGetCatalogs --- .../flight-sql/example/sqlite_server.cc | 42 +++++++++++++++++++ .../flight/flight-sql/example/sqlite_server.h | 5 +++ .../arrow/flight/flight-sql/sql_server.cpp | 4 ++ cpp/src/arrow/flight/flight-sql/sql_server.h | 8 ++++ .../flight/flight-sql/sql_server_test.cc | 37 ++++++++++++++++ 5 files changed, 96 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index cf7497bfb29..6a80aaafa08 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -111,6 +111,48 @@ Status SQLiteFlightSqlServer::DoGetStatement(const pb::sql::TicketStatementQuery return Status::OK(); } +Status GetFlightInfoForCommand(const FlightDescriptor& descriptor, + std::unique_ptr* info, + google::protobuf::Message& command, + const std::shared_ptr& schema) { + google::protobuf::Any ticketParsed; + ticketParsed.PackFrom(command); + + std::vector endpoints{ + FlightEndpoint{{ticketParsed.SerializeAsString()}, {}}}; + ARROW_ASSIGN_OR_RAISE(auto result, + FlightInfo::Make(*schema, descriptor, endpoints, -1, -1)) + + *info = std::unique_ptr(new FlightInfo(result)); + + return Status::OK(); +} + +Status SQLiteFlightSqlServer::GetFlightInfoCatalogs(const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { + pb::sql::CommandGetCatalogs command; + return GetFlightInfoForCommand(descriptor, info, command, + SqlSchema::GetCatalogsSchema()); +} + +Status SQLiteFlightSqlServer::DoGetCatalogs(const ServerCallContext& context, + std::unique_ptr* result) { + const std::shared_ptr& schema = SqlSchema::GetCatalogsSchema(); + StringBuilder catalog_name_builder; + ARROW_RETURN_NOT_OK(catalog_name_builder.Append("sqlite_master")); + + std::shared_ptr catalog_name; + ARROW_RETURN_NOT_OK(catalog_name_builder.Finish(&catalog_name)); + + const std::shared_ptr& batch = + RecordBatch::Make(schema, 1, {catalog_name}); + + ARROW_ASSIGN_OR_RAISE(auto reader, RecordBatchReader::Make({batch})); + *result = std::unique_ptr(new RecordBatchStream(reader)); + return Status::OK(); +} + } // namespace example } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index 36642ad01f2..602726dfc50 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -49,6 +49,11 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { Status DoGetStatement(const pb::sql::TicketStatementQuery &command, const ServerCallContext &context, std::unique_ptr *result) override; + Status GetFlightInfoCatalogs(const ServerCallContext &context, + const FlightDescriptor &descriptor, + std::unique_ptr *info) override; + Status DoGetCatalogs(const ServerCallContext &context, + std::unique_ptr *result) override; private: sqlite3 *db_; diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index d72c840c336..b4ffaa6a021 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -246,6 +246,10 @@ Status FlightSqlServerBase::GetFlightInfoImportedKeys( return Status::NotImplemented("DoGetExportedKeys not implemented"); } +std::shared_ptr SqlSchema::GetCatalogsSchema() { + return arrow::schema({field("catalog_name", utf8())}); +} + } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index ab2fc787ed4..d2a4cc948c6 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -258,6 +258,14 @@ class FlightSqlServerBase : public FlightServerBase { const ServerCallContext& context, std::unique_ptr* result); }; + +/// \brief Auxiliary class containing all Schemas used on Flight SQL. +class SqlSchema { + public: + + /// \brief Gets the Schema used on CommandGetCatalogs response. + static std::shared_ptr GetCatalogsSchema(); +}; } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index e6fc3742aa7..6d6e92e4805 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -112,6 +112,43 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { ASSERT_TRUE(expected_table->Equals(*table)); } +TEST(TestFlightSqlServer, TestCommandGetCatalogs) { + TestServer server("flight_sql_test_server"); + server.Start(); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + ASSERT_TRUE(server.IsRunning()); + + std::stringstream ss; + ss << "grpc://localhost:" << server.port(); + std::string uri = ss.str(); + + std::unique_ptr client; + Location location; + ASSERT_OK(Location::Parse(uri, &location)); + ASSERT_OK(FlightClient::Connect(location, &client)); + + FlightSqlClient sql_client(client); + + std::unique_ptr flight_info; + ASSERT_OK(sql_client.GetCatalogs({}, &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const std::shared_ptr& expected_schema = + arrow::schema({arrow::field("catalog_name", utf8())}); + + DECLARE_ARRAY(catalog_name_array, String, ({"sqlite_master"})); + + const std::shared_ptr
& expected_table = Table::Make( + expected_schema, {catalog_name_array}); + + ASSERT_TRUE(expected_table->Equals(*table)); +} + } // namespace sql } // namespace flight } // namespace arrow From 060e9b45998512f53d30f072693194a00c53e104 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 28 Sep 2021 16:04:32 -0300 Subject: [PATCH 034/237] Refactor tests to reduce duplication --- .../flight/flight-sql/sql_server_test.cc | 77 +++++++++---------- 1 file changed, 38 insertions(+), 39 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 6d6e92e4805..bf12be7d231 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -46,7 +46,36 @@ namespace arrow { namespace flight { namespace sql { -void ASSERT_OK(Status x) {} +TestServer* server; +FlightSqlClient* sql_client; + +class TestFlightSqlServer : public ::testing::Environment { + protected: + void SetUp() override { + server = new TestServer("flight_sql_test_server"); + server->Start(); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + ASSERT_TRUE(server->IsRunning()); + + std::stringstream ss; + ss << "grpc://localhost:" << server->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 = new FlightSqlClient(client); + } + + void TearDown() override { + server->Stop(); + + free(server); + free(sql_client); + } +}; /* * FIXME @@ -72,27 +101,11 @@ TEST(TestFlightSqlServer, FIX_PROTOBUF_BUG) { } TEST(TestFlightSqlServer, TestCommandStatementQuery) { - TestServer server("flight_sql_test_server"); - server.Start(); - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); - ASSERT_TRUE(server.IsRunning()); - - std::stringstream ss; - ss << "grpc://localhost:" << server.port(); - std::string uri = ss.str(); - - std::unique_ptr client; - Location location; - ASSERT_OK(Location::Parse(uri, &location)); - ASSERT_OK(FlightClient::Connect(location, &client)); - - FlightSqlClient sql_client(client); - std::unique_ptr flight_info; - ASSERT_OK(sql_client.Execute({}, "SELECT * FROM intTable", &flight_info)); + ASSERT_OK(sql_client->Execute({}, "SELECT * FROM intTable", &flight_info)); std::unique_ptr stream; - ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -113,27 +126,11 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { } TEST(TestFlightSqlServer, TestCommandGetCatalogs) { - TestServer server("flight_sql_test_server"); - server.Start(); - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); - ASSERT_TRUE(server.IsRunning()); - - std::stringstream ss; - ss << "grpc://localhost:" << server.port(); - std::string uri = ss.str(); - - std::unique_ptr client; - Location location; - ASSERT_OK(Location::Parse(uri, &location)); - ASSERT_OK(FlightClient::Connect(location, &client)); - - FlightSqlClient sql_client(client); - std::unique_ptr flight_info; - ASSERT_OK(sql_client.GetCatalogs({}, &flight_info)); + ASSERT_OK(sql_client->GetCatalogs({}, &flight_info)); std::unique_ptr stream; - ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -143,12 +140,14 @@ TEST(TestFlightSqlServer, TestCommandGetCatalogs) { DECLARE_ARRAY(catalog_name_array, String, ({"sqlite_master"})); - const std::shared_ptr
& expected_table = Table::Make( - expected_schema, {catalog_name_array}); + const std::shared_ptr
& expected_table = + Table::Make(expected_schema, {catalog_name_array}); ASSERT_TRUE(expected_table->Equals(*table)); } +::testing::Environment* env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); + } // namespace sql } // namespace flight } // namespace arrow From d754fdec6fbd724c50c81024d3e00a8bd7bfcd9a Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 28 Sep 2021 19:06:10 -0300 Subject: [PATCH 035/237] Fix minor comments on PR --- .../flight-sql/example/sqlite_server.cc | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 6a80aaafa08..e2cde6b5a9c 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -68,6 +68,23 @@ void SQLiteFlightSqlServer::ExecuteSql(const std::string& sql) { } } +Status GetFlightInfoForCommand(const FlightDescriptor& descriptor, + std::unique_ptr* info, + google::protobuf::Message& command, + const std::shared_ptr& schema) { + google::protobuf::Any ticketParsed; + ticketParsed.PackFrom(command); + + std::vector endpoints{ + FlightEndpoint{{ticketParsed.SerializeAsString()}, {}}}; + ARROW_ASSIGN_OR_RAISE(auto result, + FlightInfo::Make(*schema, descriptor, endpoints, -1, -1)) + + *info = std::unique_ptr(new FlightInfo(result)); + + return Status::OK(); +} + Status SQLiteFlightSqlServer::GetFlightInfoStatement( const pb::sql::CommandStatementQuery& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { @@ -111,23 +128,6 @@ Status SQLiteFlightSqlServer::DoGetStatement(const pb::sql::TicketStatementQuery return Status::OK(); } -Status GetFlightInfoForCommand(const FlightDescriptor& descriptor, - std::unique_ptr* info, - google::protobuf::Message& command, - const std::shared_ptr& schema) { - google::protobuf::Any ticketParsed; - ticketParsed.PackFrom(command); - - std::vector endpoints{ - FlightEndpoint{{ticketParsed.SerializeAsString()}, {}}}; - ARROW_ASSIGN_OR_RAISE(auto result, - FlightInfo::Make(*schema, descriptor, endpoints, -1, -1)) - - *info = std::unique_ptr(new FlightInfo(result)); - - return Status::OK(); -} - Status SQLiteFlightSqlServer::GetFlightInfoCatalogs(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { From a631b4f97f43f9f7074888c6b0e158df13bc93e5 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 28 Sep 2021 22:30:11 -0300 Subject: [PATCH 036/237] Add comment about hardcoded GetCatalogs implementation --- cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index e2cde6b5a9c..4da7f795868 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -138,6 +138,10 @@ Status SQLiteFlightSqlServer::GetFlightInfoCatalogs(const ServerCallContext& con Status SQLiteFlightSqlServer::DoGetCatalogs(const ServerCallContext& context, std::unique_ptr* result) { + // For the purpose of demonstrating a full Flight SQL server implementation, this + // returns a hard-coded catalog named "sqlite_master", although SQLite actually doesn't + // have support for catalogs. + const std::shared_ptr& schema = SqlSchema::GetCatalogsSchema(); StringBuilder catalog_name_builder; ARROW_RETURN_NOT_OK(catalog_name_builder.Append("sqlite_master")); From b8f5dda429b7aebbcc936270d3123ff0cc27f27d Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 29 Sep 2021 17:42:15 -0300 Subject: [PATCH 037/237] Make GetCatalogs return empty results --- .../arrow/flight/flight-sql/example/sqlite_server.cc | 12 ++++-------- cpp/src/arrow/flight/flight-sql/sql_server_test.cc | 8 ++------ 2 files changed, 6 insertions(+), 14 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 4da7f795868..619cbadc7a3 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -138,19 +138,15 @@ Status SQLiteFlightSqlServer::GetFlightInfoCatalogs(const ServerCallContext& con Status SQLiteFlightSqlServer::DoGetCatalogs(const ServerCallContext& context, std::unique_ptr* result) { - // For the purpose of demonstrating a full Flight SQL server implementation, this - // returns a hard-coded catalog named "sqlite_master", although SQLite actually doesn't - // have support for catalogs. + // 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_RETURN_NOT_OK(catalog_name_builder.Append("sqlite_master")); - std::shared_ptr catalog_name; - ARROW_RETURN_NOT_OK(catalog_name_builder.Finish(&catalog_name)); + StringBuilder catalog_name_builder; + ARROW_ASSIGN_OR_RAISE(auto catalog_name, catalog_name_builder.Finish()); const std::shared_ptr& batch = - RecordBatch::Make(schema, 1, {catalog_name}); + RecordBatch::Make(schema, 0, {catalog_name}); ARROW_ASSIGN_OR_RAISE(auto reader, RecordBatchReader::Make({batch})); *result = std::unique_ptr(new RecordBatchStream(reader)); diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index bf12be7d231..1be90cefee8 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -138,12 +138,8 @@ TEST(TestFlightSqlServer, TestCommandGetCatalogs) { const std::shared_ptr& expected_schema = arrow::schema({arrow::field("catalog_name", utf8())}); - DECLARE_ARRAY(catalog_name_array, String, ({"sqlite_master"})); - - const std::shared_ptr
& expected_table = - Table::Make(expected_schema, {catalog_name_array}); - - ASSERT_TRUE(expected_table->Equals(*table)); + ASSERT_TRUE(table->schema()->Equals(*expected_schema)); + ASSERT_EQ(0, table->num_rows()); } ::testing::Environment* env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); From 5e57cd15cb480bbd6f743009a6a763c8a8e26f0b Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 5 Oct 2021 13:14:55 -0300 Subject: [PATCH 038/237] Implement CommandGetSchemas --- cpp/src/arrow/flight/flight-sql/client.h | 5 +-- cpp/src/arrow/flight/flight-sql/client_impl.h | 4 +-- .../flight-sql/example/sqlite_server.cc | 32 +++++++++++++++++-- .../flight/flight-sql/example/sqlite_server.h | 7 ++++ .../arrow/flight/flight-sql/sql_server.cpp | 7 ++++ cpp/src/arrow/flight/flight-sql/sql_server.h | 3 ++ .../flight/flight-sql/sql_server_test.cc | 20 ++++++++++-- 7 files changed, 70 insertions(+), 8 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index b2b17318217..3591fce6587 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -66,8 +66,9 @@ class FlightSqlClientT { /// \param[out] flight_info The FlightInfo describing where to access the /// dataset. /// \return Status. - Status GetSchemas(const FlightCallOptions& options, std::string* catalog, - std::string* schema_filter_pattern, + Status GetSchemas(const FlightCallOptions& options, + const std::string* catalog, + const std::string* schema_filter_pattern, std::unique_ptr* flight_info) const; /// \brief Given a flight ticket and schema, request to be sent the diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index c1fbb40f43e..6bdb1af433f 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -104,8 +104,8 @@ Status FlightSqlClientT::GetCatalogs(const FlightCallOptions& options, template Status FlightSqlClientT::GetSchemas(const FlightCallOptions& options, - std::string* catalog, - std::string* schema_filter_pattern, + const std::string* catalog, + const std::string* schema_filter_pattern, std::unique_ptr* flight_info) const { pb::sql::CommandGetSchemas command; if (catalog != NULLPTR) { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 619cbadc7a3..aa7061b054b 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -18,6 +18,7 @@ #include "arrow/flight/flight-sql/example/sqlite_server.h" #include +#include #include "arrow/api.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" @@ -54,7 +55,7 @@ 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); -)"); + )"); } SQLiteFlightSqlServer::~SQLiteFlightSqlServer() { sqlite3_close(db_); } @@ -70,7 +71,7 @@ void SQLiteFlightSqlServer::ExecuteSql(const std::string& sql) { Status GetFlightInfoForCommand(const FlightDescriptor& descriptor, std::unique_ptr* info, - google::protobuf::Message& command, + const google::protobuf::Message& command, const std::shared_ptr& schema) { google::protobuf::Any ticketParsed; ticketParsed.PackFrom(command); @@ -153,6 +154,33 @@ Status SQLiteFlightSqlServer::DoGetCatalogs(const ServerCallContext& context, return Status::OK(); } +Status SQLiteFlightSqlServer::GetFlightInfoSchemas( + const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return GetFlightInfoForCommand(descriptor, info, command, + SqlSchema::GetSchemasSchema()); +} + +Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& command, + const ServerCallContext& context, + std::unique_ptr* result) { + // As SQLite doesn't support schemas, this will return an empty record batch. + + const std::shared_ptr& schema = SqlSchema::GetSchemasSchema(); + + 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})); + *result = std::unique_ptr(new RecordBatchStream(reader)); + return Status::OK(); +} + } // namespace example } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index 602726dfc50..c00d6b3de5b 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -54,6 +54,13 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { std::unique_ptr *info) override; Status DoGetCatalogs(const ServerCallContext &context, std::unique_ptr *result) override; + Status GetFlightInfoSchemas(const pb::sql::CommandGetSchemas &command, + const ServerCallContext &context, + const FlightDescriptor &descriptor, + std::unique_ptr *info) override; + Status DoGetSchemas(const pb::sql::CommandGetSchemas &command, + const ServerCallContext &context, + std::unique_ptr *result) override; private: sqlite3 *db_; diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index b4ffaa6a021..72b254b01b5 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -250,6 +250,13 @@ std::shared_ptr SqlSchema::GetCatalogsSchema() { return arrow::schema({field("catalog_name", utf8())}); } +std::shared_ptr SqlSchema::GetSchemasSchema() { + return arrow::schema({ + field("catalog_name", utf8()), + field("schema_name", utf8(), false) + }); +} + } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index d2a4cc948c6..d570b14833a 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -265,6 +265,9 @@ class SqlSchema { /// \brief Gets the Schema used on CommandGetCatalogs response. static std::shared_ptr GetCatalogsSchema(); + + /// \brief Gets the Schema used on CommandGetSchemas response. + static std::shared_ptr GetSchemasSchema(); }; } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 1be90cefee8..793e0ad5a68 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -135,8 +136,23 @@ TEST(TestFlightSqlServer, TestCommandGetCatalogs) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - const std::shared_ptr& expected_schema = - arrow::schema({arrow::field("catalog_name", utf8())}); + const std::shared_ptr& expected_schema = SqlSchema::GetCatalogsSchema(); + + ASSERT_TRUE(table->schema()->Equals(*expected_schema)); + ASSERT_EQ(0, table->num_rows()); +} + +TEST(TestFlightSqlServer, TestCommandGetSchemas) { + std::unique_ptr flight_info; + ASSERT_OK(sql_client->GetSchemas({}, NULLPTR, NULLPTR, &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + const std::shared_ptr& expected_schema = SqlSchema::GetSchemasSchema(); ASSERT_TRUE(table->schema()->Equals(*expected_schema)); ASSERT_EQ(0, table->num_rows()); From 2fdc7c84cdfc1236baaf1f2fb478e6c8048d260f Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 30 Sep 2021 13:17:10 -0300 Subject: [PATCH 039/237] Fix problem when linking protobuf to flight-sql targets --- .../arrow/flight/flight-sql/CMakeLists.txt | 13 ++++------ .../flight/flight-sql/protocol_internal.cc | 26 ------------------- .../flight/flight-sql/protocol_internal.h | 25 ------------------ .../flight/flight-sql/sql_server_test.cc | 23 ---------------- 4 files changed, 5 insertions(+), 82 deletions(-) delete mode 100644 cpp/src/arrow/flight/flight-sql/protocol_internal.cc delete mode 100644 cpp/src/arrow/flight/flight-sql/protocol_internal.h diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 3978a6a3b07..d7539079c4a 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -19,16 +19,12 @@ add_custom_target(arrow_flight_sql) arrow_install_all_headers("arrow/flight/flight-sql") -set(ARROW_FLIGHT_SQL_LINK_LIBS ${ARROW_PROTOBUF_LIBPROTOBUF}) - 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" - "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.grpc.pb.cc" - "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.grpc.pb.h") + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.h") set(PROTO_DEPENDS ${FLIGHT_SQL_PROTO} ${ARROW_PROTOBUF_LIBPROTOBUF} gRPC::grpc_cpp_plugin) @@ -51,7 +47,7 @@ set(CMAKE_CXX_FLAGS_BACKUP "${CMAKE_CXX_FLAGS}") string(REPLACE "/WX" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") string(REPLACE "-Werror " " " CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") -set(ARROW_FLIGHT_SQL_SRCS protocol_internal.cc sql_server.cpp) +set(ARROW_FLIGHT_SQL_SRCS sql_server.cpp) add_arrow_lib(arrow_flight_sql CMAKE_PACKAGE_NAME @@ -88,8 +84,8 @@ add_arrow_test(flight_sql_test SOURCES client_test.cc sql_server_test.cc + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" STATIC_LINK_LIBS - ${ARROW_FLIGHT_SQL_LINK_LIBS} ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} GTest::gtest GTest::gmock @@ -108,7 +104,8 @@ add_executable(flight_sql_test_server test_server.cc example/sqlite_statement.cc example/sqlite_statement_batch_reader.cc - example/sqlite_server.cc) + example/sqlite_server.cc + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc") target_link_libraries(flight_sql_test_server PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES} ${SQLite3_LIBRARIES}) diff --git a/cpp/src/arrow/flight/flight-sql/protocol_internal.cc b/cpp/src/arrow/flight/flight-sql/protocol_internal.cc deleted file mode 100644 index 851d6495863..00000000000 --- a/cpp/src/arrow/flight/flight-sql/protocol_internal.cc +++ /dev/null @@ -1,26 +0,0 @@ -// 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 - -#include "arrow/flight/flight-sql/protocol_internal.h" - -// NOTE(wesm): Including .cc files in another .cc file would ordinarily be a -// no-no. We have customized the serialization path for FlightData, which is -// currently only possible through some pre-processor commands that need to be -// included before either of these files is compiled. Because we don't want to -// edit the generated C++ files, we include them here and do our gRPC -// customizations in protocol-internal.h -#include "arrow/flight/flight-sql/FlightSql.grpc.pb.cc" // NOLINT -#include "arrow/flight/flight-sql/FlightSql.pb.cc" // NOLINT diff --git a/cpp/src/arrow/flight/flight-sql/protocol_internal.h b/cpp/src/arrow/flight/flight-sql/protocol_internal.h deleted file mode 100644 index 02ac1844285..00000000000 --- a/cpp/src/arrow/flight/flight-sql/protocol_internal.h +++ /dev/null @@ -1,25 +0,0 @@ -// 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 - -#pragma once - -// This header holds the Flight protobuf definitions. - -// Need to include this first to get our gRPC customizations -#include "arrow/flight/customize_protobuf.h" // IWYU pragma: export - -#include "arrow/flight/flight-sql/FlightSql.grpc.pb.h" // IWYU pragma: export -#include "arrow/flight/flight-sql/FlightSql.pb.h" // IWYU pragma: export diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 793e0ad5a68..f3336f04af4 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -78,29 +78,6 @@ class TestFlightSqlServer : public ::testing::Environment { } }; -/* - * FIXME - * For some reason when the first protobuf message used on "any.PackFrom()" has a - * string field (like CommandStatementQuery), libprotobuf throws: - * - * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3624] Invalid proto descriptor for file "google/protobuf/descriptor.proto": - * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3627] google/protobuf/descriptor.proto: Unrecognized syntax: SELECT * XXXXXX - * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3624] Invalid proto descriptor for file "FlightSql.proto": - * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3627] arrow.flight.protocol.sql.experimental: ".google.protobuf.MessageOptions" is not defined. - * [libprotobuf FATAL .../src/google/protobuf/generated_message_reflection.cc:2457] CHECK failed: file != nullptr: - * - * If the first protobuf message used on "any.PackFrom()" has no fields, all the - * subsequent calls misteriously work. - * - * Still don't know how to fix this, so we are keeping this dummy tests at the beginning - * of the test suite to move forward. - */ -TEST(TestFlightSqlServer, FIX_PROTOBUF_BUG) { - pb::sql::CommandGetCatalogs command; - google::protobuf::Any any; - any.PackFrom(command); -} - TEST(TestFlightSqlServer, TestCommandStatementQuery) { std::unique_ptr flight_info; ASSERT_OK(sql_client->Execute({}, "SELECT * FROM intTable", &flight_info)); From f1d9f9da9d297c8e17979623d32f6aeec95de654 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 30 Sep 2021 15:39:43 -0300 Subject: [PATCH 040/237] Fix checkstyle errors --- cpp/src/arrow/flight/flight-sql/CMakeLists.txt | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index d7539079c4a..3e61e8fff0f 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -22,9 +22,8 @@ arrow_install_all_headers("arrow/flight/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(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} gRPC::grpc_cpp_plugin) @@ -92,7 +91,8 @@ add_arrow_test(flight_sql_test LABELS "arrow_flight_sql") -add_executable(flight_sql_test_app test_app.cc) +add_executable(flight_sql_test_app test_app.cc + ${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc) target_link_libraries(flight_sql_test_app PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES}) From e68f6e689d366a3b2299f7c3d7e948de724735d7 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 29 Sep 2021 17:16:52 -0300 Subject: [PATCH 041/237] Implement CommandGetSchemas --- .../flight-sql/example/sqlite_statement.cc | 4 ++- .../example/sqlite_statement_batch_reader.cc | 14 ++++++++++ .../example/sqlite_statement_batch_reader.h | 9 ++++++ cpp/src/arrow/flight/protocol_internal.cc | 26 ----------------- cpp/src/arrow/flight/protocol_internal.h | 28 ------------------- 5 files changed, 26 insertions(+), 55 deletions(-) delete mode 100644 cpp/src/arrow/flight/protocol_internal.cc delete mode 100644 cpp/src/arrow/flight/protocol_internal.h diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index 66a25b2ec31..5260263e25d 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -16,6 +16,7 @@ // under the License. #include +#include #include "arrow/api.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" @@ -39,8 +40,9 @@ std::shared_ptr GetDataTypeFromSqliteType(const int column_type) { Status SqliteStatement::Create(sqlite3 *db, const std::string &sql, std::shared_ptr *result) { sqlite3_stmt *stmt; + const char *z_sql = sql.c_str(); int rc = - sqlite3_prepare_v2(db, sql.c_str(), static_cast(sql.length()), &stmt, NULLPTR); + sqlite3_prepare_v2(db, z_sql, static_cast(strlen(z_sql)), &stmt, NULLPTR); if (rc != SQLITE_OK) { sqlite3_finalize(stmt); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index 2d17373a9e7..ff4f97d7528 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -82,6 +82,16 @@ Status SqliteStatementBatchReader::Create( return Status::OK(); } +Status SqliteStatementBatchReader::Create( + const std::shared_ptr &statement_, + const std::shared_ptr &schema, + std::shared_ptr *result) { + int rc = SQLITE_OK; + result->reset(new SqliteStatementBatchReader(statement_, schema, rc)); + + return Status::OK(); +} + Status SqliteStatementBatchReader::ReadNext(std::shared_ptr *out) { sqlite3_stmt *stmt_ = statement_->GetSqlite3Stmt(); @@ -95,6 +105,10 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr *out) { ARROW_RETURN_NOT_OK(MakeBuilder(default_memory_pool(), field_type, &builders[i])); } + if (rc_ == SQLITE_OK) { + ARROW_RETURN_NOT_OK(statement_->Step(&rc_)); + } + int rows = 0; while (rows < MAX_BATCH_SIZE && rc_ == SQLITE_ROW) { rows++; diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h index 2aa80a6086f..f9b66d8860c 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h @@ -35,6 +35,15 @@ class SqliteStatementBatchReader : public RecordBatchReader { static Status Create(const std::shared_ptr &statement, std::shared_ptr *result); + /// \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. + /// \param[out] result The resulting RecordBatchReader. + /// \return Status. + static Status Create(const std::shared_ptr &statement, + const std::shared_ptr &schema, + std::shared_ptr *result); + std::shared_ptr schema() const override; Status ReadNext(std::shared_ptr *out) override; diff --git a/cpp/src/arrow/flight/protocol_internal.cc b/cpp/src/arrow/flight/protocol_internal.cc deleted file mode 100644 index 9f815398e48..00000000000 --- a/cpp/src/arrow/flight/protocol_internal.cc +++ /dev/null @@ -1,26 +0,0 @@ -// 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 - -#include "arrow/flight/protocol_internal.h" - -// NOTE(wesm): Including .cc files in another .cc file would ordinarily be a -// no-no. We have customized the serialization path for FlightData, which is -// currently only possible through some pre-processor commands that need to be -// included before either of these files is compiled. Because we don't want to -// edit the generated C++ files, we include them here and do our gRPC -// customizations in protocol-internal.h -#include "arrow/flight/Flight.grpc.pb.cc" // NOLINT -#include "arrow/flight/Flight.pb.cc" // NOLINT diff --git a/cpp/src/arrow/flight/protocol_internal.h b/cpp/src/arrow/flight/protocol_internal.h deleted file mode 100644 index 98bf9238809..00000000000 --- a/cpp/src/arrow/flight/protocol_internal.h +++ /dev/null @@ -1,28 +0,0 @@ -// 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 - -#pragma once - -// This addresses platform-specific defines, e.g. on Windows -#include "arrow/flight/platform.h" // IWYU pragma: keep - -// This header holds the Flight protobuf definitions. - -// Need to include this first to get our gRPC customizations -#include "arrow/flight/customize_protobuf.h" // IWYU pragma: export - -#include "arrow/flight/Flight.grpc.pb.h" // IWYU pragma: export -#include "arrow/flight/Flight.pb.h" // IWYU pragma: export From b22fe92358cc1a8c5d7342ae0fda7101fbd00d19 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 30 Sep 2021 11:14:06 -0300 Subject: [PATCH 042/237] Undo unscoped changes to other files --- .../flight-sql/example/sqlite_statement.cc | 4 +-- .../example/sqlite_statement_batch_reader.cc | 14 ---------- .../example/sqlite_statement_batch_reader.h | 9 ------ .../flight/flight-sql/sql_server_test.cc | 23 +++++++++++++++ cpp/src/arrow/flight/protocol_internal.cc | 26 +++++++++++++++++ cpp/src/arrow/flight/protocol_internal.h | 28 +++++++++++++++++++ 6 files changed, 78 insertions(+), 26 deletions(-) create mode 100644 cpp/src/arrow/flight/protocol_internal.cc create mode 100644 cpp/src/arrow/flight/protocol_internal.h diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index 5260263e25d..66a25b2ec31 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -16,7 +16,6 @@ // under the License. #include -#include #include "arrow/api.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" @@ -40,9 +39,8 @@ std::shared_ptr GetDataTypeFromSqliteType(const int column_type) { Status SqliteStatement::Create(sqlite3 *db, const std::string &sql, std::shared_ptr *result) { sqlite3_stmt *stmt; - const char *z_sql = sql.c_str(); int rc = - sqlite3_prepare_v2(db, z_sql, static_cast(strlen(z_sql)), &stmt, NULLPTR); + sqlite3_prepare_v2(db, sql.c_str(), static_cast(sql.length()), &stmt, NULLPTR); if (rc != SQLITE_OK) { sqlite3_finalize(stmt); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index ff4f97d7528..2d17373a9e7 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -82,16 +82,6 @@ Status SqliteStatementBatchReader::Create( return Status::OK(); } -Status SqliteStatementBatchReader::Create( - const std::shared_ptr &statement_, - const std::shared_ptr &schema, - std::shared_ptr *result) { - int rc = SQLITE_OK; - result->reset(new SqliteStatementBatchReader(statement_, schema, rc)); - - return Status::OK(); -} - Status SqliteStatementBatchReader::ReadNext(std::shared_ptr *out) { sqlite3_stmt *stmt_ = statement_->GetSqlite3Stmt(); @@ -105,10 +95,6 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr *out) { ARROW_RETURN_NOT_OK(MakeBuilder(default_memory_pool(), field_type, &builders[i])); } - if (rc_ == SQLITE_OK) { - ARROW_RETURN_NOT_OK(statement_->Step(&rc_)); - } - int rows = 0; while (rows < MAX_BATCH_SIZE && rc_ == SQLITE_ROW) { rows++; diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h index f9b66d8860c..2aa80a6086f 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h @@ -35,15 +35,6 @@ class SqliteStatementBatchReader : public RecordBatchReader { static Status Create(const std::shared_ptr &statement, std::shared_ptr *result); - /// \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. - /// \param[out] result The resulting RecordBatchReader. - /// \return Status. - static Status Create(const std::shared_ptr &statement, - const std::shared_ptr &schema, - std::shared_ptr *result); - std::shared_ptr schema() const override; Status ReadNext(std::shared_ptr *out) override; diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index f3336f04af4..793e0ad5a68 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -78,6 +78,29 @@ class TestFlightSqlServer : public ::testing::Environment { } }; +/* + * FIXME + * For some reason when the first protobuf message used on "any.PackFrom()" has a + * string field (like CommandStatementQuery), libprotobuf throws: + * + * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3624] Invalid proto descriptor for file "google/protobuf/descriptor.proto": + * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3627] google/protobuf/descriptor.proto: Unrecognized syntax: SELECT * XXXXXX + * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3624] Invalid proto descriptor for file "FlightSql.proto": + * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3627] arrow.flight.protocol.sql.experimental: ".google.protobuf.MessageOptions" is not defined. + * [libprotobuf FATAL .../src/google/protobuf/generated_message_reflection.cc:2457] CHECK failed: file != nullptr: + * + * If the first protobuf message used on "any.PackFrom()" has no fields, all the + * subsequent calls misteriously work. + * + * Still don't know how to fix this, so we are keeping this dummy tests at the beginning + * of the test suite to move forward. + */ +TEST(TestFlightSqlServer, FIX_PROTOBUF_BUG) { + pb::sql::CommandGetCatalogs command; + google::protobuf::Any any; + any.PackFrom(command); +} + TEST(TestFlightSqlServer, TestCommandStatementQuery) { std::unique_ptr flight_info; ASSERT_OK(sql_client->Execute({}, "SELECT * FROM intTable", &flight_info)); diff --git a/cpp/src/arrow/flight/protocol_internal.cc b/cpp/src/arrow/flight/protocol_internal.cc new file mode 100644 index 00000000000..9f815398e48 --- /dev/null +++ b/cpp/src/arrow/flight/protocol_internal.cc @@ -0,0 +1,26 @@ +// 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 + +#include "arrow/flight/protocol_internal.h" + +// NOTE(wesm): Including .cc files in another .cc file would ordinarily be a +// no-no. We have customized the serialization path for FlightData, which is +// currently only possible through some pre-processor commands that need to be +// included before either of these files is compiled. Because we don't want to +// edit the generated C++ files, we include them here and do our gRPC +// customizations in protocol-internal.h +#include "arrow/flight/Flight.grpc.pb.cc" // NOLINT +#include "arrow/flight/Flight.pb.cc" // NOLINT diff --git a/cpp/src/arrow/flight/protocol_internal.h b/cpp/src/arrow/flight/protocol_internal.h new file mode 100644 index 00000000000..98bf9238809 --- /dev/null +++ b/cpp/src/arrow/flight/protocol_internal.h @@ -0,0 +1,28 @@ +// 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 + +#pragma once + +// This addresses platform-specific defines, e.g. on Windows +#include "arrow/flight/platform.h" // IWYU pragma: keep + +// This header holds the Flight protobuf definitions. + +// Need to include this first to get our gRPC customizations +#include "arrow/flight/customize_protobuf.h" // IWYU pragma: export + +#include "arrow/flight/Flight.grpc.pb.h" // IWYU pragma: export +#include "arrow/flight/Flight.pb.h" // IWYU pragma: export From c750b15a1c1d7543517f82bb3fe2594ad06d9e5b Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 6 Oct 2021 15:11:16 -0300 Subject: [PATCH 043/237] Fix style issues --- cpp/src/arrow/flight/flight-sql/client.h | 3 +- cpp/src/arrow/flight/flight-sql/client_impl.h | 3 +- .../arrow/flight/flight-sql/client_test.cc | 76 +++++++++---------- .../flight-sql/example/sqlite_server.cc | 1 + .../flight/flight-sql/example/sqlite_server.h | 44 +++++------ .../flight-sql/example/sqlite_statement.cc | 38 +++++----- .../flight-sql/example/sqlite_statement.h | 14 ++-- .../example/sqlite_statement_batch_reader.cc | 21 +++-- .../example/sqlite_statement_batch_reader.h | 10 +-- .../arrow/flight/flight-sql/sql_server.cpp | 46 ++++++----- cpp/src/arrow/flight/flight-sql/sql_server.h | 1 - .../flight/flight-sql/sql_server_test.cc | 26 +------ cpp/src/arrow/flight/flight-sql/test_app.cc | 62 ++++++--------- .../arrow/flight/flight-sql/test_server.cc | 4 +- 14 files changed, 152 insertions(+), 197 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 3591fce6587..45bfaae1136 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -66,8 +66,7 @@ class FlightSqlClientT { /// \param[out] flight_info The FlightInfo describing where to access the /// dataset. /// \return Status. - Status GetSchemas(const FlightCallOptions& options, - const std::string* catalog, + Status GetSchemas(const FlightCallOptions& options, const std::string* catalog, const std::string* schema_filter_pattern, std::unique_ptr* flight_info) const; diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 6bdb1af433f..16f2d040f72 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -67,8 +67,7 @@ Status FlightSqlClientT::Execute(const FlightCallOptions& options, template Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, - const std::string& query, - int64_t* rows) const { + const std::string& query, int64_t* rows) const { pb::sql::CommandStatementUpdate command; command.set_query(query); diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 1f5173b9a36..105411953b2 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -37,12 +37,12 @@ class FlightClientMock { public: MOCK_METHOD(Status, GetFlightInfo, (const FlightCallOptions&, const FlightDescriptor&, - std::unique_ptr*)); + std::unique_ptr*)); MOCK_METHOD(Status, DoPut, (const FlightCallOptions&, const FlightDescriptor&, - const std::shared_ptr& schema, - std::unique_ptr*, - std::unique_ptr*)); + const std::shared_ptr& schema, + std::unique_ptr*, + std::unique_ptr*)); }; class FlightMetadataReaderMock : public FlightMetadataReader { @@ -68,7 +68,7 @@ FlightDescriptor getDescriptor(google::protobuf::Message& command) { } TEST(TestFlightSqlClient, TestGetCatalogs) { - auto *client_mock = new FlightClientMock(); + auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; @@ -77,14 +77,13 @@ TEST(TestFlightSqlClient, TestGetCatalogs) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, - GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.GetCatalogs(call_options, &flight_info); + (void)sqlClient.GetCatalogs(call_options, &flight_info); } TEST(TestFlightSqlClient, TestGetSchemas) { - auto *client_mock = new FlightClientMock(); + auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; @@ -98,15 +97,14 @@ TEST(TestFlightSqlClient, TestGetSchemas) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, - GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.GetSchemas(call_options, &catalog, - &schema_filter_pattern, &flight_info); + (void)sqlClient.GetSchemas(call_options, &catalog, &schema_filter_pattern, + &flight_info); } TEST(TestFlightSqlClient, TestGetTables) { - auto *client_mock = new FlightClientMock(); + auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; @@ -122,22 +120,21 @@ TEST(TestFlightSqlClient, TestGetTables) { command.set_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) { + for (const std::string& table_type : table_types) { command.add_table_types(table_type); } FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, - GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.GetTables(call_options, &catalog, &schema_filter_pattern, - &table_name_filter_pattern, include_schema, - table_types, &flight_info); + (void)sqlClient.GetTables(call_options, &catalog, &schema_filter_pattern, + &table_name_filter_pattern, include_schema, table_types, + &flight_info); } TEST(TestFlightSqlClient, TestGetTableTypes) { - auto *client_mock = new FlightClientMock(); + auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; @@ -146,14 +143,13 @@ TEST(TestFlightSqlClient, TestGetTableTypes) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, - GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.GetTableTypes(call_options, &flight_info); + (void)sqlClient.GetTableTypes(call_options, &flight_info); } TEST(TestFlightSqlClient, TestGetExported) { - auto *client_mock = new FlightClientMock(); + auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; @@ -169,14 +165,13 @@ TEST(TestFlightSqlClient, TestGetExported) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, - GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.GetExportedKeys(call_options, &catalog, &schema, table, &flight_info); + (void)sqlClient.GetExportedKeys(call_options, &catalog, &schema, table, &flight_info); } TEST(TestFlightSqlClient, TestGetImported) { - auto *client_mock = new FlightClientMock(); + auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; @@ -192,14 +187,13 @@ TEST(TestFlightSqlClient, TestGetImported) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, - GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.GetImportedKeys(call_options, &catalog, &schema, table, &flight_info); + (void)sqlClient.GetImportedKeys(call_options, &catalog, &schema, table, &flight_info); } TEST(TestFlightSqlClient, TestGetPrimary) { - auto *client_mock = new FlightClientMock(); + auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; @@ -215,14 +209,13 @@ TEST(TestFlightSqlClient, TestGetPrimary) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, - GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.GetPrimaryKeys(call_options, &catalog, &schema, table, &flight_info); + (void)sqlClient.GetPrimaryKeys(call_options, &catalog, &schema, table, &flight_info); } TEST(TestFlightSqlClient, TestExecute) { - auto *client_mock = new FlightClientMock(); + auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; @@ -234,14 +227,13 @@ TEST(TestFlightSqlClient, TestExecute) { FlightDescriptor descriptor = getDescriptor(command); std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, - GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sqlClient.Execute(call_options, query, &flight_info); + (void)sqlClient.Execute(call_options, query, &flight_info); } TEST(TestFlightSqlClient, TestExecuteUpdate) { - auto *client_mock = new FlightClientMock(); + auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; @@ -282,7 +274,7 @@ TEST(TestFlightSqlClient, TestExecuteUpdate) { int64_t num_rows; - (void) sqlClient.ExecuteUpdate(call_options, query, &num_rows); + (void)sqlClient.ExecuteUpdate(call_options, query, &num_rows); ASSERT_EQ(num_rows, 100); } diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index aa7061b054b..b874ea3580a 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -18,6 +18,7 @@ #include "arrow/flight/flight-sql/example/sqlite_server.h" #include + #include #include "arrow/api.h" diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index c00d6b3de5b..35f8a81ab02 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -20,9 +20,9 @@ #include #include "arrow/api.h" -#include "arrow/flight/flight-sql/sql_server.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/flight-sql/sql_server.h" namespace arrow { namespace flight { @@ -39,31 +39,31 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { /// \brief Auxiliary method used to execute an arbitrary SQL statement on the underlying /// SQLite database. - void ExecuteSql(const std::string &sql); + void ExecuteSql(const std::string& sql); - Status GetFlightInfoStatement(const pb::sql::CommandStatementQuery &command, - const ServerCallContext &context, - const FlightDescriptor &descriptor, - std::unique_ptr *info) override; + Status GetFlightInfoStatement(const pb::sql::CommandStatementQuery& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) override; - Status DoGetStatement(const pb::sql::TicketStatementQuery &command, - const ServerCallContext &context, - std::unique_ptr *result) override; - Status GetFlightInfoCatalogs(const ServerCallContext &context, - const FlightDescriptor &descriptor, - std::unique_ptr *info) override; - Status DoGetCatalogs(const ServerCallContext &context, - std::unique_ptr *result) override; - Status GetFlightInfoSchemas(const pb::sql::CommandGetSchemas &command, - const ServerCallContext &context, - const FlightDescriptor &descriptor, - std::unique_ptr *info) override; - Status DoGetSchemas(const pb::sql::CommandGetSchemas &command, - const ServerCallContext &context, - std::unique_ptr *result) override; + Status DoGetStatement(const pb::sql::TicketStatementQuery& command, + const ServerCallContext& context, + std::unique_ptr* result) override; + Status GetFlightInfoCatalogs(const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) override; + Status DoGetCatalogs(const ServerCallContext& context, + std::unique_ptr* result) override; + Status GetFlightInfoSchemas(const pb::sql::CommandGetSchemas& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) override; + Status DoGetSchemas(const pb::sql::CommandGetSchemas& command, + const ServerCallContext& context, + std::unique_ptr* result) override; private: - sqlite3 *db_; + sqlite3* db_; }; } // namespace example diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index 66a25b2ec31..248a4e8ffda 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. +#include "arrow/flight/flight-sql/example/sqlite_statement.h" + #include #include "arrow/api.h" -#include "arrow/flight/flight-sql/example/sqlite_statement.h" namespace arrow { namespace flight { @@ -27,18 +28,23 @@ 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_INTEGER: + return int64(); + case SQLITE_FLOAT: + return float64(); + case SQLITE_BLOB: + return binary(); + case SQLITE_TEXT: + return utf8(); case SQLITE_NULL: - default:return null(); + default: + return null(); } } -Status SqliteStatement::Create(sqlite3 *db, const std::string &sql, - std::shared_ptr *result) { - sqlite3_stmt *stmt; +Status SqliteStatement::Create(sqlite3* db, const std::string& sql, + std::shared_ptr* result) { + sqlite3_stmt* stmt; int rc = sqlite3_prepare_v2(db, sql.c_str(), static_cast(sql.length()), &stmt, NULLPTR); @@ -51,11 +57,11 @@ Status SqliteStatement::Create(sqlite3 *db, const std::string &sql, return Status::OK(); } -Status SqliteStatement::GetSchema(std::shared_ptr *schema) const { +Status SqliteStatement::GetSchema(std::shared_ptr* schema) 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); + const char* column_name = sqlite3_column_name(stmt_, i); const int column_type = sqlite3_column_type(stmt_, i); std::shared_ptr data_type = GetDataTypeFromSqliteType(column_type); @@ -66,11 +72,9 @@ Status SqliteStatement::GetSchema(std::shared_ptr *schema) const { return Status::OK(); } -SqliteStatement::~SqliteStatement() { - sqlite3_finalize(stmt_); -} +SqliteStatement::~SqliteStatement() { sqlite3_finalize(stmt_); } -Status SqliteStatement::Step(int *rc) { +Status SqliteStatement::Step(int* rc) { *rc = sqlite3_step(stmt_); if (*rc == SQLITE_ERROR) { return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db_)); @@ -79,9 +83,7 @@ Status SqliteStatement::Step(int *rc) { return Status::OK(); } -sqlite3_stmt *SqliteStatement::GetSqlite3Stmt() { - return stmt_; -} +sqlite3_stmt* SqliteStatement::GetSqlite3Stmt() { return stmt_; } } // namespace example } // namespace sql diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h index a0108dbffe6..7d32de342f8 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h @@ -33,15 +33,15 @@ class SqliteStatement { /// \param[in] sql SQL statement. /// \param[out] result The resulting SqliteStatement. /// \return Status. - static Status Create(sqlite3 *db, const std::string &sql, - std::shared_ptr *result); + static Status Create(sqlite3* db, const std::string& sql, + std::shared_ptr* result); ~SqliteStatement(); /// \brief Creates an Arrow Schema based on the results of this statement. /// \param[out] schema The resulting Schema. /// \return Status. - Status GetSchema(std::shared_ptr *schema) const; + Status GetSchema(std::shared_ptr* schema) const; /// \brief Steps on underlying sqlite3_stmt. /// \param[out] rc The resulting return code from SQLite. @@ -49,13 +49,13 @@ class SqliteStatement { Status Step(int* rc); /// \brief Returns the underlying sqlite3_stmt. - sqlite3_stmt *GetSqlite3Stmt(); + sqlite3_stmt* GetSqlite3Stmt(); private: - sqlite3 *db_; - sqlite3_stmt *stmt_; + sqlite3* db_; + sqlite3_stmt* stmt_; - SqliteStatement(sqlite3 *db, sqlite3_stmt *stmt) : db_(db), stmt_(stmt) {} + SqliteStatement(sqlite3* db, sqlite3_stmt* stmt) : db_(db), stmt_(stmt) {} }; } // namespace example diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index 2d17373a9e7..c67117d178a 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -15,12 +15,12 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" #include #include "arrow/api.h" +#include "arrow/flight/flight-sql/example/sqlite_statement.h" #define STRING_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ case TYPE_CLASS##Type::type_id: { \ @@ -68,9 +68,8 @@ SqliteStatementBatchReader::SqliteStatementBatchReader( : statement_(std::move(statement)), schema_(std::move(schema)), rc_(rc) {} Status SqliteStatementBatchReader::Create( - const std::shared_ptr &statement_, - std::shared_ptr *result) { - + const std::shared_ptr& statement_, + std::shared_ptr* result) { int rc; ARROW_RETURN_NOT_OK(statement_->Step(&rc)); @@ -82,15 +81,15 @@ Status SqliteStatementBatchReader::Create( return Status::OK(); } -Status SqliteStatementBatchReader::ReadNext(std::shared_ptr *out) { - sqlite3_stmt *stmt_ = statement_->GetSqlite3Stmt(); +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(); + 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])); } @@ -99,9 +98,9 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr *out) { while (rows < MAX_BATCH_SIZE && 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]; + const std::shared_ptr& field = schema_->field(i); + const std::shared_ptr& field_type = field->type(); + ArrayBuilder& builder = *builders[i]; switch (field_type->id()) { INT_BUILDER_CASE(Int64, stmt_, i) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h index 2aa80a6086f..9b221f8ecb3 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h @@ -20,6 +20,7 @@ #include #include "arrow/api.h" +#include "arrow/flight/flight-sql/example/sqlite_statement.h" namespace arrow { namespace flight { @@ -32,12 +33,12 @@ class SqliteStatementBatchReader : public RecordBatchReader { /// \param[in] statement SQLite statement to be read. /// \param[out] result The resulting RecordBatchReader. /// \return Status. - static Status Create(const std::shared_ptr &statement, - std::shared_ptr *result); + static Status Create(const std::shared_ptr& statement, + std::shared_ptr* result); std::shared_ptr schema() const override; - Status ReadNext(std::shared_ptr *out) override; + Status ReadNext(std::shared_ptr* out) override; private: std::shared_ptr statement_; @@ -45,8 +46,7 @@ class SqliteStatementBatchReader : public RecordBatchReader { int rc_; SqliteStatementBatchReader(std::shared_ptr statement, - std::shared_ptr schema, - int rc); + std::shared_ptr schema, int rc); }; } // namespace example diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index 72b254b01b5..87aa6da2e98 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -32,43 +32,43 @@ Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, if (any.Is()) { pb::sql::CommandStatementQuery command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoStatement(command, context, request, info)); + return GetFlightInfoStatement(command, context, request, info); } else if (any.Is()) { pb::sql::CommandPreparedStatementQuery command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoPreparedStatement(command, context, request, info)); + return GetFlightInfoPreparedStatement(command, context, request, info); } else if (any.Is()) { pb::sql::CommandGetCatalogs command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoCatalogs(context, request, info)); + return GetFlightInfoCatalogs(context, request, info); } else if (any.Is()) { pb::sql::CommandGetSchemas command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoSchemas(command, context, request, info)); + return GetFlightInfoSchemas(command, context, request, info); } else if (any.Is()) { pb::sql::CommandGetTables command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoTables(command, context, request, info)); + return GetFlightInfoTables(command, context, request, info); } else if (any.Is()) { pb::sql::CommandGetTableTypes command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoTableTypes(context, request, info)); + return GetFlightInfoTableTypes(context, request, info); } else if (any.Is()) { pb::sql::CommandGetSqlInfo command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoSqlInfo(command, context, request, info)); + return GetFlightInfoSqlInfo(command, context, request, info); } else if (any.Is()) { pb::sql::CommandGetPrimaryKeys command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoPrimaryKeys(command, context, request, info)); + return GetFlightInfoPrimaryKeys(command, context, request, info); } else if (any.Is()) { pb::sql::CommandGetExportedKeys command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoExportedKeys(command, context, request, info)); + return GetFlightInfoExportedKeys(command, context, request, info); } else if (any.Is()) { pb::sql::CommandGetImportedKeys command; any.UnpackTo(&command); - ARROW_RETURN_NOT_OK(GetFlightInfoImportedKeys(command, context, request, info)); + return GetFlightInfoImportedKeys(command, context, request, info); } return Status::Invalid("The defined request is invalid."); @@ -84,43 +84,43 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket if (anyCommand.Is()) { pb::sql::TicketStatementQuery command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetStatement(command, context, stream)); + return DoGetStatement(command, context, stream); } else if (anyCommand.Is()) { pb::sql::CommandPreparedStatementQuery command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetPreparedStatement(command, context, stream)); + return DoGetPreparedStatement(command, context, stream); } else if (anyCommand.Is()) { pb::sql::CommandGetCatalogs command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetCatalogs(context, stream)); + return DoGetCatalogs(context, stream); } else if (anyCommand.Is()) { pb::sql::CommandGetSchemas command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetSchemas(command, context, stream)); + return DoGetSchemas(command, context, stream); } else if (anyCommand.Is()) { pb::sql::CommandGetTables command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetTables(command, context, stream)); + return DoGetTables(command, context, stream); } else if (anyCommand.Is()) { pb::sql::CommandGetTableTypes command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetTableTypes(context, stream)); + return DoGetTableTypes(context, stream); } else if (anyCommand.Is()) { pb::sql::CommandGetSqlInfo command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetSqlInfo(command, context, stream)); + return DoGetSqlInfo(command, context, stream); } else if (anyCommand.Is()) { pb::sql::CommandGetPrimaryKeys command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetPrimaryKeys(command, context, stream)); + return DoGetPrimaryKeys(command, context, stream); } else if (anyCommand.Is()) { pb::sql::CommandGetExportedKeys command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetExportedKeys(command, context, stream)); + return DoGetExportedKeys(command, context, stream); } else if (anyCommand.Is()) { pb::sql::CommandGetImportedKeys command; anyCommand.UnpackTo(&command); - ARROW_RETURN_NOT_OK(DoGetImportedKeys(command, context, stream)); + return DoGetImportedKeys(command, context, stream); } return Status::Invalid("The defined request is invalid."); @@ -251,10 +251,8 @@ std::shared_ptr SqlSchema::GetCatalogsSchema() { } std::shared_ptr SqlSchema::GetSchemasSchema() { - return arrow::schema({ - field("catalog_name", utf8()), - field("schema_name", utf8(), false) - }); + return arrow::schema( + {field("catalog_name", utf8()), field("schema_name", utf8(), false)}); } } // namespace sql diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index d570b14833a..7d7a8b1365f 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -262,7 +262,6 @@ class FlightSqlServerBase : public FlightServerBase { /// \brief Auxiliary class containing all Schemas used on Flight SQL. class SqlSchema { public: - /// \brief Gets the Schema used on CommandGetCatalogs response. static std::shared_ptr GetCatalogsSchema(); diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 793e0ad5a68..795af7cf1e2 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -78,29 +79,6 @@ class TestFlightSqlServer : public ::testing::Environment { } }; -/* - * FIXME - * For some reason when the first protobuf message used on "any.PackFrom()" has a - * string field (like CommandStatementQuery), libprotobuf throws: - * - * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3624] Invalid proto descriptor for file "google/protobuf/descriptor.proto": - * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3627] google/protobuf/descriptor.proto: Unrecognized syntax: SELECT * XXXXXX - * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3624] Invalid proto descriptor for file "FlightSql.proto": - * [libprotobuf ERROR .../src/google/protobuf/descriptor.cc:3627] arrow.flight.protocol.sql.experimental: ".google.protobuf.MessageOptions" is not defined. - * [libprotobuf FATAL .../src/google/protobuf/generated_message_reflection.cc:2457] CHECK failed: file != nullptr: - * - * If the first protobuf message used on "any.PackFrom()" has no fields, all the - * subsequent calls misteriously work. - * - * Still don't know how to fix this, so we are keeping this dummy tests at the beginning - * of the test suite to move forward. - */ -TEST(TestFlightSqlServer, FIX_PROTOBUF_BUG) { - pb::sql::CommandGetCatalogs command; - google::protobuf::Any any; - any.PackFrom(command); -} - TEST(TestFlightSqlServer, TestCommandStatementQuery) { std::unique_ptr flight_info; ASSERT_OK(sql_client->Execute({}, "SELECT * FROM intTable", &flight_info)); @@ -158,7 +136,7 @@ TEST(TestFlightSqlServer, TestCommandGetSchemas) { ASSERT_EQ(0, table->num_rows()); } -::testing::Environment* env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); +auto env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index 9d602cbc69a..29ba11020a3 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -15,16 +15,17 @@ // specific language governing permissions and limitations // under the License. -#include -#include #include #include +#include #include #include #include -#include #include +#include +#include + using arrow::Result; using arrow::Schema; using arrow::Status; @@ -51,13 +52,13 @@ DEFINE_string(catalog, "", "Catalog"); DEFINE_string(schema, "", "Schema"); DEFINE_string(table, "", "Table"); -Status PrintResultsForEndpoint(const FlightSqlClient &client, - const FlightCallOptions &call_options, - const FlightEndpoint &endpoint) { +Status PrintResultsForEndpoint(const FlightSqlClient& client, + const FlightCallOptions& call_options, + const FlightEndpoint& endpoint) { std::unique_ptr stream; ARROW_RETURN_NOT_OK(client.DoGet(call_options, endpoint.ticket, &stream)); - const arrow::Result> &schema = stream->GetSchema(); + const arrow::Result>& schema = stream->GetSchema(); ARROW_RETURN_NOT_OK(schema); std::cout << "Schema:" << std::endl; @@ -82,10 +83,9 @@ Status PrintResultsForEndpoint(const FlightSqlClient &client, return Status::OK(); } -Status PrintResults(FlightSqlClient &client, - const FlightCallOptions &call_options, - const std::unique_ptr &info) { - const std::vector &endpoints = info->endpoints(); +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() @@ -106,7 +106,7 @@ Status RunMain() { if (!fLS::FLAGS_username.empty() || !fLS::FLAGS_password.empty()) { Result> bearer_result = - client->AuthenticateBasicToken({}, fLS::FLAGS_username, fLS::FLAGS_password); + client->AuthenticateBasicToken({}, fLS::FLAGS_username, fLS::FLAGS_password); ARROW_RETURN_NOT_OK(bearer_result); call_options.headers.push_back(bearer_result.ValueOrDie()); @@ -126,42 +126,30 @@ Status RunMain() { std::unique_ptr info; if (fLS::FLAGS_command == "Execute") { - ARROW_RETURN_NOT_OK(sqlClient.Execute(call_options, - fLS::FLAGS_query, &info)); + ARROW_RETURN_NOT_OK(sqlClient.Execute(call_options, fLS::FLAGS_query, &info)); } else if (fLS::FLAGS_command == "GetCatalogs") { ARROW_RETURN_NOT_OK(sqlClient.GetCatalogs(call_options, &info)); } else if (fLS::FLAGS_command == "GetSchemas") { - ARROW_RETURN_NOT_OK(sqlClient.GetSchemas(call_options, - &fLS::FLAGS_catalog, - &fLS::FLAGS_schema, - &info)); + ARROW_RETURN_NOT_OK(sqlClient.GetSchemas(call_options, &fLS::FLAGS_catalog, + &fLS::FLAGS_schema, &info)); } else if (fLS::FLAGS_command == "GetTableTypes") { ARROW_RETURN_NOT_OK(sqlClient.GetTableTypes(call_options, &info)); } else if (fLS::FLAGS_command == "GetTables") { std::vector table_types = {}; bool include_schema = false; - ARROW_RETURN_NOT_OK(sqlClient.GetTables(call_options, - &fLS::FLAGS_catalog, - &fLS::FLAGS_schema, - &fLS::FLAGS_table, - include_schema, - table_types, &info)); + ARROW_RETURN_NOT_OK(sqlClient.GetTables(call_options, &fLS::FLAGS_catalog, + &fLS::FLAGS_schema, &fLS::FLAGS_table, + include_schema, table_types, &info)); } else if (fLS::FLAGS_command == "GetExportedKeys") { - ARROW_RETURN_NOT_OK(sqlClient.GetExportedKeys(call_options, - &fLS::FLAGS_catalog, - &fLS::FLAGS_schema, - fLS::FLAGS_table, &info)); + ARROW_RETURN_NOT_OK(sqlClient.GetExportedKeys( + call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, fLS::FLAGS_table, &info)); } else if (fLS::FLAGS_command == "GetImportedKeys") { - ARROW_RETURN_NOT_OK(sqlClient.GetImportedKeys(call_options, - &fLS::FLAGS_catalog, - &fLS::FLAGS_schema, - fLS::FLAGS_table, &info)); + ARROW_RETURN_NOT_OK(sqlClient.GetImportedKeys( + call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, fLS::FLAGS_table, &info)); } else if (fLS::FLAGS_command == "GetPrimaryKeys") { - ARROW_RETURN_NOT_OK(sqlClient.GetPrimaryKeys(call_options, - &fLS::FLAGS_catalog, - &fLS::FLAGS_schema, - fLS::FLAGS_table, &info)); + ARROW_RETURN_NOT_OK(sqlClient.GetPrimaryKeys( + call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, fLS::FLAGS_table, &info)); } if (info != NULLPTR) { @@ -171,7 +159,7 @@ Status RunMain() { return Status::OK(); } -int main(int argc, char **argv) { +int main(int argc, char** argv) { gflags::ParseCommandLineFlags(&argc, &argv, true); Status st = RunMain(); diff --git a/cpp/src/arrow/flight/flight-sql/test_server.cc b/cpp/src/arrow/flight/flight-sql/test_server.cc index c0f8725f757..9e89f332670 100644 --- a/cpp/src/arrow/flight/flight-sql/test_server.cc +++ b/cpp/src/arrow/flight/flight-sql/test_server.cc @@ -21,13 +21,13 @@ #include #include +#include "arrow/flight/flight-sql/example/sqlite_server.h" #include "arrow/flight/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" -#include "arrow/flight/flight-sql/example/sqlite_server.h" DEFINE_int32(port, 31337, "Server port to listen on"); @@ -35,7 +35,7 @@ namespace pb = arrow::flight::protocol; std::unique_ptr g_server; -int main(int argc, char **argv) { +int main(int argc, char** argv) { gflags::SetUsageMessage("Integration testing server for Flight SQL."); gflags::ParseCommandLineFlags(&argc, &argv, true); From cac2d9fb56408e223be9ea0ca76582e57a0f2223 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 6 Oct 2021 15:11:16 -0300 Subject: [PATCH 044/237] Fix style issues --- cpp/src/arrow/flight/flight-sql/sql_server_test.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 795af7cf1e2..4201ae44d5c 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -26,6 +26,7 @@ #include #include #include +#include #define unparen(...) __VA_ARGS__ #define DECLARE_ARRAY(ARRAY_NAME, TYPE_CLASS, DATA) \ From 203e64f1116072e06874430be839832824965f04 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 28 Sep 2021 16:08:46 -0300 Subject: [PATCH 045/237] Create a class to deal the DoGetTables when schema is included --- .../SqliteTablesWithSchemaBatchreader.cpp | 97 +++++++++++++++++++ .../SqliteTablesWithSchemaBatchreader.h | 36 +++++++ 2 files changed, 133 insertions(+) create mode 100644 cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.cpp create mode 100644 cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.h diff --git a/cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.cpp b/cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.cpp new file mode 100644 index 00000000000..88b7973ef74 --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.cpp @@ -0,0 +1,97 @@ +// +// Created by jose on 9/24/21. +// + +#include "SqliteTablesWithSchemaBatchreader.h" + +#include +#include +#include +#include + +#include "arrow/flight/flight-sql/example/sqlite_statement.h" + +namespace arrow { +namespace flight { +namespace sql { + +std::shared_ptr SqliteTablesWithSchemaBatchReader::schema() const { + return SqlSchema::GetTablesSchemaWithSchema(); +} + +Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* batch) { + const std::string schema_query = + "SELECT table_name, name, type, [notnull] FROM pragma_table_info(table_name)" + " JOIN (SELECT tbl_name as table_name FROM sqlite_master)"; + + std::shared_ptr schema_statement; + ARROW_RETURN_NOT_OK( + example::SqliteStatement::Create(db_, schema_query, &schema_statement)); + + 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()); + + for (int i = 0; i < table_name_array->length(); i++) { + const std::string& table_name = string_array->GetString(i); + std::vector> column_fields; + + while(sqlite3_step(schema_statement-> GetSqlite3Stmt()) == SQLITE_ROW) { + const char* string1 = + reinterpret_cast(sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 0)); + std::string string = std::string(string1); + if (string == table_name) { + const char* column_name = (char*)sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 1); + const char* column_type = (char*) 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)); + ARROW_RETURN_NOT_OK( + schema_builder.Append(value.ValueOrDie()->data(), value.ValueOrDie()->size())); + } + + std::shared_ptr schema_array; + ARROW_RETURN_NOT_OK(schema_builder.Finish(&schema_array)); + + auto result = first_batch->AddColumn(4, "table_schema", schema_array); + + ARROW_ASSIGN_OR_RAISE(*batch, result); + + return Status::OK(); +} + +std::shared_ptr SqliteTablesWithSchemaBatchReader::GetArrowType(const std::string& sqlite_type) { + if(sqlite_type == "INT") { + return int64(); + } else if ( sqlite_type == "REAL") { + return float64(); + } else if(sqlite_type == "BLOB") { + return binary(); + } else if (sqlite_type == "TEXT" || sqlite_type.find("CHAR") != std::string::npos) { + return utf8(); + } else { + return null(); + } + } +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.h b/cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.h new file mode 100644 index 00000000000..f8ab4eb566d --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.h @@ -0,0 +1,36 @@ +// +// Created by jose on 9/24/21. +// + +#ifndef ARROW_SQLITETABLESWITHSCHEMABATCHREADER_H +#define ARROW_SQLITETABLESWITHSCHEMABATCHREADER_H + +#include +#include +#include "arrow/flight/flight-sql/example/sqlite_statement.h" +#include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" + +namespace arrow { +namespace flight { +namespace sql { + +class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { + public: + std::shared_ptr reader_; + sqlite3* db_; + + SqliteTablesWithSchemaBatchReader(std::shared_ptr reader, + sqlite3* db_) + : reader_(reader), db_(db_) {} + + std::shared_ptr schema() const override; + + Status ReadNext(std::shared_ptr* batch) override; + + std::shared_ptr GetArrowType(const std::string& sqlite_type); +}; +} // namespace sql +} // namespace flight +} // namespace arrow + +#endif // ARROW_SQLITETABLESWITHSCHEMABATCHREADER_H From 6b6d9c2b9bc13d2a46b13dce81f9f16f723fbaf8 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 28 Sep 2021 16:09:09 -0300 Subject: [PATCH 046/237] Include methods GetTablesFlightInfo and DoGetTables --- .../flight-sql/example/sqlite_server.cc | 58 +++++++++++++++++++ .../flight/flight-sql/example/sqlite_server.h | 7 ++- 2 files changed, 64 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index b874ea3580a..303f59d0c33 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -25,6 +25,7 @@ #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" #include "arrow/flight/flight-sql/sql_server.h" +#include "SqliteTablesWithSchemaBatchreader.h" namespace arrow { namespace flight { @@ -182,6 +183,63 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& com return Status::OK(); } + Status + SQLiteFlightSqlServer::GetFlightInfoTables(const pb::sql::CommandGetTables &command, const ServerCallContext &context, + const FlightDescriptor &descriptor, std::unique_ptr *info) { + google::protobuf::Any ticketParsed; + + ticketParsed.PackFrom(command); + + std::vector endpoints{ + FlightEndpoint{{ticketParsed.SerializeAsString()}, {}}}; + + if (command.include_schema()) { + ARROW_ASSIGN_OR_RAISE(auto result, + FlightInfo::Make(*SqlSchema::GetTablesSchemaWithSchema(), + descriptor, endpoints, -1, -1)) + *info = std::unique_ptr(new FlightInfo(result)); + } else { + ARROW_ASSIGN_OR_RAISE(auto result, FlightInfo::Make(*SqlSchema::GetTablesSchema(), + descriptor, endpoints, -1, -1)) + *info = std::unique_ptr(new FlightInfo(result)); + } + + return Status::OK(); + } + + Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables &command, const ServerCallContext &context, + std::unique_ptr *result) { + std::string table_query( + "SELECT 'sqlite_master' as catalog_name, 'main' as schema_name, name as " + "table_name, type as table_type FROM sqlite_master where 1=1"); + + std::shared_ptr batch_reader; + + if (command.has_catalog()) { + table_query = table_query + " and catalog_name='" + command.catalog() + "'"; + } + + if (command.has_schema_filter_pattern()) { + table_query = + table_query + " and schema_name LIKE '" + command.schema_filter_pattern() + "'"; + } + + std::shared_ptr statement; + ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, table_query, &statement)); + + std::shared_ptr reader; + ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create(statement, &reader)); + + if (command.include_schema()) { + std::shared_ptr pReader = + std::make_shared(reader, db_); + *result = std::unique_ptr(new RecordBatchStream(pReader)); + } else { + *result = std::unique_ptr(new RecordBatchStream(reader)); + } + + return Status::OK(); + } } // namespace example } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index 35f8a81ab02..ead7169bd63 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -62,7 +62,12 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { const ServerCallContext& context, std::unique_ptr* result) override; - private: + Status GetFlightInfoTables(const pb::sql::CommandGetTables &command, const ServerCallContext &context, + const FlightDescriptor &descriptor, std::unique_ptr *info) override; + + Status DoGetTables(const pb::sql::CommandGetTables &command, const ServerCallContext &context, + std::unique_ptr *result) override; +private: sqlite3* db_; }; From 86a0ba0b3d1a186dab33c48ffedaa1ebbbcfbf63 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 28 Sep 2021 16:09:27 -0300 Subject: [PATCH 047/237] Add the schemas to be used by the GetTabkes --- cpp/src/arrow/flight/flight-sql/sql_server.cpp | 11 +++++++++++ cpp/src/arrow/flight/flight-sql/sql_server.h | 2 ++ 2 files changed, 13 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index 87aa6da2e98..608e984f54c 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -255,6 +255,17 @@ std::shared_ptr SqlSchema::GetSchemasSchema() { {field("catalog_name", utf8()), field("schema_name", utf8(), false)}); } +std::shared_ptr SqlSchema::GetTablesSchema() { + return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8()), + field("table_name", utf8()), field("table_type", utf8())}); +} + +std::shared_ptr SqlSchema::GetTablesSchemaWithSchema() { + return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8()), + field("table_name", utf8()), field("table_type", utf8()), + field("table_schema", binary())}); +} + } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 7d7a8b1365f..bcf4755c61e 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -267,6 +267,8 @@ class SqlSchema { /// \brief Gets the Schema used on CommandGetSchemas response. static std::shared_ptr GetSchemasSchema(); + static std::shared_ptr GetTablesSchema(); + static std::shared_ptr GetTablesSchemaWithSchema(); }; } // namespace sql } // namespace flight From 7d53dd6a3343569fca9279e9968c5f7678895fcc Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 28 Sep 2021 16:09:39 -0300 Subject: [PATCH 048/237] Add class to tge CMakeLists.txt --- cpp/src/arrow/flight/flight-sql/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 3e61e8fff0f..f9aa8888357 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -105,6 +105,7 @@ add_executable(flight_sql_test_server example/sqlite_statement.cc example/sqlite_statement_batch_reader.cc example/sqlite_server.cc + example/SqliteTablesWithSchemaBatchreader.cpp "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc") target_link_libraries(flight_sql_test_server PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES} From 814dc10579876ef44017a9bb74eec0efef224fd7 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 29 Sep 2021 11:28:23 -0300 Subject: [PATCH 049/237] Refactor the name of class sqlite table schema batch reader --- cpp/src/arrow/flight/flight-sql/CMakeLists.txt | 2 +- ...atchreader.cpp => sqlite_tables_schema_batch_reader.cpp} | 2 +- ...emaBatchreader.h => sqlite_tables_schema_batch_reader.h} | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) rename cpp/src/arrow/flight/flight-sql/example/{SqliteTablesWithSchemaBatchreader.cpp => sqlite_tables_schema_batch_reader.cpp} (98%) rename cpp/src/arrow/flight/flight-sql/example/{SqliteTablesWithSchemaBatchreader.h => sqlite_tables_schema_batch_reader.h} (85%) diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index f9aa8888357..7334445dab2 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -105,7 +105,7 @@ add_executable(flight_sql_test_server example/sqlite_statement.cc example/sqlite_statement_batch_reader.cc example/sqlite_server.cc - example/SqliteTablesWithSchemaBatchreader.cpp + example/sqlite_tables_schema_batch_reader.cpp "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc") target_link_libraries(flight_sql_test_server PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES} diff --git a/cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.cpp b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp similarity index 98% rename from cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.cpp rename to cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp index 88b7973ef74..2f1886e8030 100644 --- a/cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.cpp +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp @@ -2,7 +2,7 @@ // Created by jose on 9/24/21. // -#include "SqliteTablesWithSchemaBatchreader.h" +#include "sqlite_tables_schema_batch_reader.h" #include #include diff --git a/cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h similarity index 85% rename from cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.h rename to cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h index f8ab4eb566d..7e985323400 100644 --- a/cpp/src/arrow/flight/flight-sql/example/SqliteTablesWithSchemaBatchreader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h @@ -2,8 +2,8 @@ // Created by jose on 9/24/21. // -#ifndef ARROW_SQLITETABLESWITHSCHEMABATCHREADER_H -#define ARROW_SQLITETABLESWITHSCHEMABATCHREADER_H +#ifndef ARROW_SQLITE_TABLES_SCHEMA_BATCH_READER_H +#define ARROW_SQLITE_TABLES_SCHEMA_BATCH_READER_H #include #include @@ -33,4 +33,4 @@ class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { } // namespace flight } // namespace arrow -#endif // ARROW_SQLITETABLESWITHSCHEMABATCHREADER_H +#endif // ARROW_SQLITE_TABLES_SCHEMA_BATCH_READER_H From aef712a73a4640c76088e08cd8ca339b934e16e0 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 29 Sep 2021 11:28:41 -0300 Subject: [PATCH 050/237] Add new filter to the query on GetTables --- cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 303f59d0c33..397c59cda73 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -25,7 +25,7 @@ #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" #include "arrow/flight/flight-sql/sql_server.h" -#include "SqliteTablesWithSchemaBatchreader.h" +#include "sqlite_tables_schema_batch_reader.h" namespace arrow { namespace flight { @@ -224,6 +224,10 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& com table_query + " and schema_name LIKE '" + command.schema_filter_pattern() + "'"; } + if (command.has_table_name_filter_pattern()) { + table_query = table_query + " and table_name LIKE '" + command.table_name_filter_pattern() + "'"; + } + std::shared_ptr statement; ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, table_query, &statement)); From 8a3dc6d8e20cf8826849453607dae92b83d6050d Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 29 Sep 2021 11:28:54 -0300 Subject: [PATCH 051/237] Add new tests to GetTables --- .../flight/flight-sql/sql_server_test.cc | 270 ++++++++++++++++++ 1 file changed, 270 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 4201ae44d5c..5349bb6cb8f 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -26,6 +26,7 @@ #include #include #include +#include #include #define unparen(...) __VA_ARGS__ @@ -40,6 +41,15 @@ ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ } +#define DECLARE_EMPTY_ARRAY(ARRAY_NAME, TYPE_CLASS) \ + std::shared_ptr ARRAY_NAME; \ + { \ + arrow::TYPE_CLASS##Builder builder; \ + builder.AppendNull(); \ + \ + ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ + } + using ::testing::_; using ::testing::Ref; @@ -105,6 +115,266 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { ASSERT_TRUE(expected_table->Equals(*table)); } +TEST(TestFlightSqlServer, TestCommandGetTables) { + TestServer server("flight_sql_test_server"); + server.Start(); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + ASSERT_TRUE(server.IsRunning()); + + std::stringstream ss; + ss << "grpc://localhost:" << server.port(); + std::string uri = ss.str(); + + std::unique_ptr client; + Location location; + ASSERT_OK(Location::Parse(uri, &location)); + ASSERT_OK(FlightClient::Connect(location, &client)); + + FlightSqlClient sql_client(client); + + std::unique_ptr flight_info; + std::vector table_types; + ASSERT_OK(sql_client.GetTables({}, nullptr, nullptr, nullptr, false, table_types, &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master"})); + DECLARE_ARRAY(schema_name, String, ({"main", "main", "main", "main", "main" })); + DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable", "COMPANY", "sqlite_autoindex_COMPANY_1"})); + DECLARE_ARRAY(table_type, String, ({"table", "table", "table", "table", "index"})); + + const std::shared_ptr
& expected_table = Table::Make( + SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); + + ASSERT_TRUE(expected_table->Equals(*table)); +} + +TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { + TestServer server("flight_sql_test_server"); + server.Start(); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + ASSERT_TRUE(server.IsRunning()); + + std::stringstream ss; + ss << "grpc://localhost:" << server.port(); + std::string uri = ss.str(); + + std::unique_ptr client; + Location location; + ASSERT_OK(Location::Parse(uri, &location)); + ASSERT_OK(FlightClient::Connect(location, &client)); + + FlightSqlClient sql_client(client); + + std::unique_ptr flight_info; + std::vector table_types; + + std::string table_filter_pattern = "COMPA%"; + ASSERT_OK(sql_client.GetTables({}, nullptr, nullptr, &table_filter_pattern, false, table_types, &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + DECLARE_ARRAY(catalog_name, String, ({"sqlite_master"})); + DECLARE_ARRAY(schema_name, String, ({"main" })); + DECLARE_ARRAY(table_name, String, ({"COMPANY"})); + DECLARE_ARRAY(table_type, String, ({"table"})); + + const std::shared_ptr
& expected_table = Table::Make( + SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); + + ASSERT_TRUE(expected_table->Equals(*table)); +} + +TEST(TestFlightSqlServer, TestCommandGetTablesWithSchemaFilter) { + TestServer server("flight_sql_test_server"); + server.Start(); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + ASSERT_TRUE(server.IsRunning()); + + std::stringstream ss; + ss << "grpc://localhost:" << server.port(); + std::string uri = ss.str(); + + std::unique_ptr client; + Location location; + ASSERT_OK(Location::Parse(uri, &location)); + ASSERT_OK(FlightClient::Connect(location, &client)); + + FlightSqlClient sql_client(client); + + std::unique_ptr flight_info; + std::vector table_types; + + std::string schema_filter_pattern = "main"; + ASSERT_OK(sql_client.GetTables({}, nullptr, &schema_filter_pattern, nullptr, false, table_types, &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master"})); + DECLARE_ARRAY(schema_name, String, ({"main", "main", "main", "main", "main" })); + DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable", "COMPANY", "sqlite_autoindex_COMPANY_1"})); + DECLARE_ARRAY(table_type, String, ({"table", "table", "table", "table", "index"})); + + const std::shared_ptr
& expected_table = Table::Make( + SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); + + ASSERT_TRUE(expected_table->Equals(*table)); +} + +TEST(TestFlightSqlServer, TestCommandGetTablesWithCatalogFilter) { + TestServer server("flight_sql_test_server"); + server.Start(); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + ASSERT_TRUE(server.IsRunning()); + + std::stringstream ss; + ss << "grpc://localhost:" << server.port(); + std::string uri = ss.str(); + + std::unique_ptr client; + Location location; + ASSERT_OK(Location::Parse(uri, &location)); + ASSERT_OK(FlightClient::Connect(location, &client)); + + FlightSqlClient sql_client(client); + + std::unique_ptr flight_info; + std::vector table_types; + + std::string catalog_filter_pattern = "sqlite_master"; + ASSERT_OK(sql_client.GetTables({}, &catalog_filter_pattern, nullptr, nullptr, false, table_types, &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master"})); + DECLARE_ARRAY(schema_name, String, ({"main", "main", "main", "main", "main" })); + DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable", "COMPANY", "sqlite_autoindex_COMPANY_1"})); + DECLARE_ARRAY(table_type, String, ({"table", "table", "table", "table", "index"})); + + const std::shared_ptr
& expected_table = Table::Make( + SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); + + ASSERT_TRUE(expected_table->Equals(*table)); +} + + +TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceSchemaFilter) { + TestServer server("flight_sql_test_server"); + server.Start(); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + ASSERT_TRUE(server.IsRunning()); + + std::stringstream ss; + ss << "grpc://localhost:" << server.port(); + std::string uri = ss.str(); + + std::unique_ptr client; + Location location; + ASSERT_OK(Location::Parse(uri, &location)); + ASSERT_OK(FlightClient::Connect(location, &client)); + + FlightSqlClient sql_client(client); + + std::unique_ptr flight_info; + std::vector table_types; + + std::string schema_filter_pattern = "unknown"; + ASSERT_OK(sql_client.GetTables({}, nullptr, &schema_filter_pattern, nullptr, false, table_types, &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master"})); + DECLARE_ARRAY(schema_name, String, ({"main", "main", "main", "main", "main" })); + DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable", "COMPANY", "sqlite_autoindex_COMPANY_1"})); + DECLARE_ARRAY(table_type, String, ({"table", "table", "table", "table", "index"})); + + const std::shared_ptr
& expected_table = Table::Make( + SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); + + ASSERT_TRUE(expected_table->Equals(*table)); +} + +TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceCatalogFilter) { + TestServer server("flight_sql_test_server"); + server.Start(); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + ASSERT_TRUE(server.IsRunning()); + + std::stringstream ss; + ss << "grpc://localhost:" << server.port(); + std::string uri = ss.str(); + + std::unique_ptr client; + Location location; + ASSERT_OK(Location::Parse(uri, &location)); + ASSERT_OK(FlightClient::Connect(location, &client)); + + FlightSqlClient sql_client(client); + + std::unique_ptr flight_info; + std::vector table_types; + + std::string catalog_filter_pattern = "unknown"; + ASSERT_OK(sql_client.GetTables({}, &catalog_filter_pattern, nullptr, nullptr, false, table_types, &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + StringBuilder catalog_name_builder; + StringBuilder schema_name_builder; + StringBuilder table_name_builder; + StringBuilder table_type_builder; + + std::shared_ptr catalog_name; + std::shared_ptr schema_name; + std::shared_ptr table_name; + std::shared_ptr table_type; + + catalog_name_builder.Finish(&catalog_name); + schema_name_builder.Finish(&schema_name); + table_name_builder.Finish(&table_name); + table_type_builder.Finish(&table_type); + +// DECLARE_EMPTY_ARRAY(catalog_name, String); +// DECLARE_EMPTY_ARRAY(schema_name, String); +// DECLARE_EMPTY_ARRAY(table_name, String); +// DECLARE_EMPTY_ARRAY(table_type, String); + + std::unique_ptr builder; + MakeBuilder(default_memory_pool(), utf8(), &builder); + builder->Resize(0); + builder->Finish(); + + const std::shared_ptr
& expected_table = Table::Make( + SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}, 0); + + + ASSERT_TRUE(expected_table->Equals(*table)); +} + TEST(TestFlightSqlServer, TestCommandGetCatalogs) { std::unique_ptr flight_info; ASSERT_OK(sql_client->GetCatalogs({}, &flight_info)); From a0f32ba9ab4bf390110d772c353592f018d5f692 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 09:59:53 -0300 Subject: [PATCH 052/237] Refactor test from GetTables --- .../flight/flight-sql/sql_server_test.cc | 160 ++++-------------- 1 file changed, 30 insertions(+), 130 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 5349bb6cb8f..e84a80362c6 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -116,36 +116,20 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { } TEST(TestFlightSqlServer, TestCommandGetTables) { - TestServer server("flight_sql_test_server"); - server.Start(); - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); - ASSERT_TRUE(server.IsRunning()); - - std::stringstream ss; - ss << "grpc://localhost:" << server.port(); - std::string uri = ss.str(); - - std::unique_ptr client; - Location location; - ASSERT_OK(Location::Parse(uri, &location)); - ASSERT_OK(FlightClient::Connect(location, &client)); - - FlightSqlClient sql_client(client); - std::unique_ptr flight_info; std::vector table_types; - ASSERT_OK(sql_client.GetTables({}, nullptr, nullptr, nullptr, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types, &flight_info)); std::unique_ptr stream; - ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master"})); - DECLARE_ARRAY(schema_name, String, ({"main", "main", "main", "main", "main" })); - DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable", "COMPANY", "sqlite_autoindex_COMPANY_1"})); - DECLARE_ARRAY(table_type, String, ({"table", "table", "table", "table", "index"})); + DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master"})); + DECLARE_ARRAY(schema_name, String, ({"main", "main", "main"})); + DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); + DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); const std::shared_ptr
& expected_table = Table::Make( SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); @@ -154,37 +138,22 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { } TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { - TestServer server("flight_sql_test_server"); - server.Start(); - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); - ASSERT_TRUE(server.IsRunning()); - - std::stringstream ss; - ss << "grpc://localhost:" << server.port(); - std::string uri = ss.str(); - - std::unique_ptr client; - Location location; - ASSERT_OK(Location::Parse(uri, &location)); - ASSERT_OK(FlightClient::Connect(location, &client)); - - FlightSqlClient sql_client(client); - std::unique_ptr flight_info; std::vector table_types; - std::string table_filter_pattern = "COMPA%"; - ASSERT_OK(sql_client.GetTables({}, nullptr, nullptr, &table_filter_pattern, false, table_types, &flight_info)); + std::string table_filter_pattern = "int%"; + std::string schema_filter_pattern = "main"; + ASSERT_OK(sql_client->GetTables({}, nullptr, &schema_filter_pattern, &table_filter_pattern, false, table_types, &flight_info)); std::unique_ptr stream; - ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); DECLARE_ARRAY(catalog_name, String, ({"sqlite_master"})); DECLARE_ARRAY(schema_name, String, ({"main" })); - DECLARE_ARRAY(table_name, String, ({"COMPANY"})); + DECLARE_ARRAY(table_name, String, ({"intTable"})); DECLARE_ARRAY(table_type, String, ({"table"})); const std::shared_ptr
& expected_table = Table::Make( @@ -194,38 +163,22 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { } TEST(TestFlightSqlServer, TestCommandGetTablesWithSchemaFilter) { - TestServer server("flight_sql_test_server"); - server.Start(); - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); - ASSERT_TRUE(server.IsRunning()); - - std::stringstream ss; - ss << "grpc://localhost:" << server.port(); - std::string uri = ss.str(); - - std::unique_ptr client; - Location location; - ASSERT_OK(Location::Parse(uri, &location)); - ASSERT_OK(FlightClient::Connect(location, &client)); - - FlightSqlClient sql_client(client); - std::unique_ptr flight_info; std::vector table_types; std::string schema_filter_pattern = "main"; - ASSERT_OK(sql_client.GetTables({}, nullptr, &schema_filter_pattern, nullptr, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, nullptr, &schema_filter_pattern, nullptr, false, table_types, &flight_info)); std::unique_ptr stream; - ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master"})); - DECLARE_ARRAY(schema_name, String, ({"main", "main", "main", "main", "main" })); - DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable", "COMPANY", "sqlite_autoindex_COMPANY_1"})); - DECLARE_ARRAY(table_type, String, ({"table", "table", "table", "table", "index"})); + DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master"})); + DECLARE_ARRAY(schema_name, String, ({"main", "main", "main"})); + DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); + DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); const std::shared_ptr
& expected_table = Table::Make( SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); @@ -234,38 +187,23 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithSchemaFilter) { } TEST(TestFlightSqlServer, TestCommandGetTablesWithCatalogFilter) { - TestServer server("flight_sql_test_server"); - server.Start(); - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); - ASSERT_TRUE(server.IsRunning()); - - std::stringstream ss; - ss << "grpc://localhost:" << server.port(); - std::string uri = ss.str(); - - std::unique_ptr client; - Location location; - ASSERT_OK(Location::Parse(uri, &location)); - ASSERT_OK(FlightClient::Connect(location, &client)); - - FlightSqlClient sql_client(client); - std::unique_ptr flight_info; std::vector table_types; std::string catalog_filter_pattern = "sqlite_master"; - ASSERT_OK(sql_client.GetTables({}, &catalog_filter_pattern, nullptr, nullptr, false, table_types, &flight_info)); + std::string table_filter_pattern = "int%"; + ASSERT_OK(sql_client->GetTables({}, &catalog_filter_pattern, nullptr, &table_filter_pattern, false, table_types, &flight_info)); std::unique_ptr stream; - ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master"})); - DECLARE_ARRAY(schema_name, String, ({"main", "main", "main", "main", "main" })); - DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable", "COMPANY", "sqlite_autoindex_COMPANY_1"})); - DECLARE_ARRAY(table_type, String, ({"table", "table", "table", "table", "index"})); + DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master"})); + DECLARE_ARRAY(schema_name, String, ({"main", "main", "main"})); + DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); + DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); const std::shared_ptr
& expected_table = Table::Make( SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); @@ -275,70 +213,32 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithCatalogFilter) { TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceSchemaFilter) { - TestServer server("flight_sql_test_server"); - server.Start(); - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); - ASSERT_TRUE(server.IsRunning()); - - std::stringstream ss; - ss << "grpc://localhost:" << server.port(); - std::string uri = ss.str(); - - std::unique_ptr client; - Location location; - ASSERT_OK(Location::Parse(uri, &location)); - ASSERT_OK(FlightClient::Connect(location, &client)); - - FlightSqlClient sql_client(client); - std::unique_ptr flight_info; std::vector table_types; std::string schema_filter_pattern = "unknown"; - ASSERT_OK(sql_client.GetTables({}, nullptr, &schema_filter_pattern, nullptr, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, nullptr, &schema_filter_pattern, nullptr, false, table_types, &flight_info)); std::unique_ptr stream; - ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master", "sqlite_master"})); - DECLARE_ARRAY(schema_name, String, ({"main", "main", "main", "main", "main" })); - DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable", "COMPANY", "sqlite_autoindex_COMPANY_1"})); - DECLARE_ARRAY(table_type, String, ({"table", "table", "table", "table", "index"})); + ASSERT_TRUE(table->schema()->Equals(SqlSchema::GetTablesSchema())); - const std::shared_ptr
& expected_table = Table::Make( - SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); - - ASSERT_TRUE(expected_table->Equals(*table)); + ASSERT_EQ(table->num_rows(), 0); } TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceCatalogFilter) { - TestServer server("flight_sql_test_server"); - server.Start(); - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); - ASSERT_TRUE(server.IsRunning()); - - std::stringstream ss; - ss << "grpc://localhost:" << server.port(); - std::string uri = ss.str(); - - std::unique_ptr client; - Location location; - ASSERT_OK(Location::Parse(uri, &location)); - ASSERT_OK(FlightClient::Connect(location, &client)); - - FlightSqlClient sql_client(client); - std::unique_ptr flight_info; std::vector table_types; std::string catalog_filter_pattern = "unknown"; - ASSERT_OK(sql_client.GetTables({}, &catalog_filter_pattern, nullptr, nullptr, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, &catalog_filter_pattern, nullptr, nullptr, false, table_types, &flight_info)); std::unique_ptr stream; - ASSERT_OK(sql_client.DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); From 8a02752d9f1ef897937e0c48f98c42213f2f8672 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 15:07:45 -0300 Subject: [PATCH 053/237] Refactor methods that parse the table type to field type --- .../example/sqlite_tables_schema_batch_reader.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp index 2f1886e8030..d23171c17c6 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include "arrow/flight/flight-sql/example/sqlite_statement.h" @@ -80,13 +81,15 @@ Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* } std::shared_ptr SqliteTablesWithSchemaBatchReader::GetArrowType(const std::string& sqlite_type) { - if(sqlite_type == "INT") { + if(boost::iequals(sqlite_type, "int") || boost::iequals(sqlite_type, "integer")) { return int64(); - } else if ( sqlite_type == "REAL") { + } else if ( boost::iequals(sqlite_type, "REAL")) { return float64(); - } else if(sqlite_type == "BLOB") { + } else if(boost::iequals(sqlite_type, "BLOB")) { return binary(); - } else if (sqlite_type == "TEXT" || sqlite_type.find("CHAR") != std::string::npos) { + } else if (boost::iequals(sqlite_type, "TEXT") || + boost::icontains(sqlite_type, "char") || + boost::icontains(sqlite_type, "varchar")) { return utf8(); } else { return null(); From 01266d72b56edc38071b83bfcabc8257be277cfc Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 15:08:23 -0300 Subject: [PATCH 054/237] Add table type filter to the query --- .../flight/flight-sql/example/sqlite_server.cc | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 397c59cda73..d8fd7a49b82 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -228,11 +228,25 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& com table_query = table_query + " and table_name LIKE '" + command.table_name_filter_pattern() + "'"; } + if(command.table_types_size() > 0) { + google::protobuf::RepeatedPtrField types = command.table_types(); + + std::stringstream ss; + int size = types.size(); + for (int i = 0; i < size; i++) { + ss << "'" << types.at(i) << "'"; + if(size - 1 != i){ + ss << ","; + } + } + table_query = table_query + " and table_type IN (" + ss.str() + ")"; + } + std::shared_ptr statement; ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, table_query, &statement)); std::shared_ptr reader; - ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create(statement, &reader)); + ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create(statement, SqlSchema::GetTablesSchema(), &reader)); if (command.include_schema()) { std::shared_ptr pReader = From 5bc0e9f2449cad3f128a936194682c7e52f9b8cf Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 15:08:57 -0300 Subject: [PATCH 055/237] Add a DECLARE_BINARY_ARRAY for testing --- .../arrow/flight/flight-sql/sql_server_test.cc | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index e84a80362c6..423019471d3 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -41,14 +41,16 @@ ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ } -#define DECLARE_EMPTY_ARRAY(ARRAY_NAME, TYPE_CLASS) \ - std::shared_ptr ARRAY_NAME; \ - { \ - arrow::TYPE_CLASS##Builder builder; \ - builder.AppendNull(); \ - \ - ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ - } +#define DECLARE_BINARY_ARRAY(ARRAY_NAME, DATA, LENGTH) \ +std::shared_ptr ARRAY_NAME; \ +{ \ +arrow::Binary##Builder builder; \ +auto data = unparen DATA; \ +for (const auto& item : data) { \ +ASSERT_OK(builder.Append(item, LENGTH)); \ +} \ +ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ +} using ::testing::_; using ::testing::Ref; From 60b8326c29411d709dcf43b4b2aca9277d8baa99 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 15:09:11 -0300 Subject: [PATCH 056/237] Add more test for the GetTables --- .../flight/flight-sql/sql_server_test.cc | 99 ++++++++++++++----- 1 file changed, 73 insertions(+), 26 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 423019471d3..d384da842d8 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -144,8 +144,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { std::vector table_types; std::string table_filter_pattern = "int%"; - std::string schema_filter_pattern = "main"; - ASSERT_OK(sql_client->GetTables({}, nullptr, &schema_filter_pattern, &table_filter_pattern, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, false, table_types, &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); @@ -193,8 +192,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithCatalogFilter) { std::vector table_types; std::string catalog_filter_pattern = "sqlite_master"; - std::string table_filter_pattern = "int%"; - ASSERT_OK(sql_client->GetTables({}, &catalog_filter_pattern, nullptr, &table_filter_pattern, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, &catalog_filter_pattern, nullptr, nullptr, false, table_types, &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); @@ -213,7 +211,6 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithCatalogFilter) { ASSERT_TRUE(expected_table->Equals(*table)); } - TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceSchemaFilter) { std::unique_ptr flight_info; std::vector table_types; @@ -245,34 +242,84 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceCatalogFilter) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - StringBuilder catalog_name_builder; - StringBuilder schema_name_builder; - StringBuilder table_name_builder; - StringBuilder table_type_builder; + ASSERT_TRUE(table->schema()->Equals(SqlSchema::GetTablesSchema())); - std::shared_ptr catalog_name; - std::shared_ptr schema_name; - std::shared_ptr table_name; - std::shared_ptr table_type; + ASSERT_EQ(table->num_rows(), 0); +} - catalog_name_builder.Finish(&catalog_name); - schema_name_builder.Finish(&schema_name); - table_name_builder.Finish(&table_name); - table_type_builder.Finish(&table_type); +TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { + std::unique_ptr flight_info; + std::vector table_types {"index"}; -// DECLARE_EMPTY_ARRAY(catalog_name, String); -// DECLARE_EMPTY_ARRAY(schema_name, String); -// DECLARE_EMPTY_ARRAY(table_name, String); -// DECLARE_EMPTY_ARRAY(table_type, String); + ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types, &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + ASSERT_TRUE(table->schema()->Equals(SqlSchema::GetTablesSchema())); + + ASSERT_EQ(table->num_rows(), 0); +} + +TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { + std::unique_ptr flight_info; + std::vector table_types {"table"}; + + ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types, &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); - std::unique_ptr builder; - MakeBuilder(default_memory_pool(), utf8(), &builder); - builder->Resize(0); - builder->Finish(); + DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master"})); + DECLARE_ARRAY(schema_name, String, ({"main", "main", "main"})); + DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); + DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); const std::shared_ptr
& expected_table = Table::Make( - SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}, 0); + SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); + ASSERT_TRUE(expected_table->Equals(*table)); +} + +TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { + std::unique_ptr flight_info; + std::vector table_types; + + std::string table_filter_pattern = "int%"; + ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, true, table_types, &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + DECLARE_ARRAY(catalog_name, String, ({"sqlite_master"})); + DECLARE_ARRAY(schema_name, String, ({"main" })); + DECLARE_ARRAY(table_name, String, ({"intTable"})); + DECLARE_ARRAY(table_type, String, ({"table"})); + + const std::shared_ptr schema_table = arrow::schema( + {arrow::field("id", int64(), true, NULL), + arrow::field("keyName", utf8(), true, NULL), + arrow::field("value", int64(), true, NULL), + arrow::field("foreignId", int64(), true, NULL) + }); + + const arrow::Result>& value = + ipc::SerializeSchema(*schema_table); + value.ValueOrDie()->data(), value.ValueOrDie()->size(); + + DECLARE_BINARY_ARRAY(table_schema, ({value.ValueOrDie()->data()}), value.ValueOrDie()->size()); + + const std::shared_ptr
& expected_table = Table::Make( + SqlSchema::GetTablesSchemaWithSchema(), {catalog_name, schema_name, table_name, table_type, table_schema}); ASSERT_TRUE(expected_table->Equals(*table)); } From 8e846c84631e6dcf2dce99d129e8b788c12aae57 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 15:39:18 -0300 Subject: [PATCH 057/237] Fix checkstyle --- .../flight-sql/example/sqlite_server.cc | 44 ++++----- .../flight/flight-sql/example/sqlite_server.h | 14 +-- .../sqlite_tables_schema_batch_reader.cpp | 53 +++++++---- .../sqlite_tables_schema_batch_reader.h | 18 +++- .../arrow/flight/flight-sql/sql_server.cpp | 2 +- cpp/src/arrow/flight/flight-sql/sql_server.h | 8 +- .../flight/flight-sql/sql_server_test.cc | 90 +++++++++++-------- 7 files changed, 143 insertions(+), 86 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index d8fd7a49b82..a604a400d1b 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -15,17 +15,16 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight-sql/example/sqlite_server.h" - #include #include #include "arrow/api.h" +#include "arrow/flight/flight-sql/sql_server.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" -#include "arrow/flight/flight-sql/sql_server.h" -#include "sqlite_tables_schema_batch_reader.h" +#include "arrow/flight/flight-sql/example/sqlite_server.h" +#include "arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h" namespace arrow { namespace flight { @@ -183,32 +182,33 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& com return Status::OK(); } - Status - SQLiteFlightSqlServer::GetFlightInfoTables(const pb::sql::CommandGetTables &command, const ServerCallContext &context, - const FlightDescriptor &descriptor, std::unique_ptr *info) { +Status SQLiteFlightSqlServer::GetFlightInfoTables( + const pb::sql::CommandGetTables& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { google::protobuf::Any ticketParsed; ticketParsed.PackFrom(command); std::vector endpoints{ - FlightEndpoint{{ticketParsed.SerializeAsString()}, {}}}; + FlightEndpoint{{ticketParsed.SerializeAsString()}, {}}}; if (command.include_schema()) { - ARROW_ASSIGN_OR_RAISE(auto result, - FlightInfo::Make(*SqlSchema::GetTablesSchemaWithSchema(), - descriptor, endpoints, -1, -1)) - *info = std::unique_ptr(new FlightInfo(result)); + ARROW_ASSIGN_OR_RAISE( + auto result, FlightInfo::Make(*SqlSchema::GetTablesSchemaWithIncludedSchema(), + descriptor, endpoints, -1, -1)) + *info = std::unique_ptr(new FlightInfo(result)); } else { ARROW_ASSIGN_OR_RAISE(auto result, FlightInfo::Make(*SqlSchema::GetTablesSchema(), descriptor, endpoints, -1, -1)) - *info = std::unique_ptr(new FlightInfo(result)); + *info = std::unique_ptr(new FlightInfo(result)); } return Status::OK(); - } +} - Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables &command, const ServerCallContext &context, - std::unique_ptr *result) { +Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& command, + const ServerCallContext& context, + std::unique_ptr* result) { std::string table_query( "SELECT 'sqlite_master' as catalog_name, 'main' as schema_name, name as " "table_name, type as table_type FROM sqlite_master where 1=1"); @@ -225,17 +225,18 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& com } if (command.has_table_name_filter_pattern()) { - table_query = table_query + " and table_name LIKE '" + command.table_name_filter_pattern() + "'"; + table_query = table_query + " and table_name LIKE '" + + command.table_name_filter_pattern() + "'"; } - if(command.table_types_size() > 0) { + if (command.table_types_size() > 0) { google::protobuf::RepeatedPtrField types = command.table_types(); std::stringstream ss; int size = types.size(); for (int i = 0; i < size; i++) { ss << "'" << types.at(i) << "'"; - if(size - 1 != i){ + if (size - 1 != i) { ss << ","; } } @@ -246,7 +247,8 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& com ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, table_query, &statement)); std::shared_ptr reader; - ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create(statement, SqlSchema::GetTablesSchema(), &reader)); + ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create( + statement, SqlSchema::GetTablesSchema(), &reader)); if (command.include_schema()) { std::shared_ptr pReader = @@ -257,7 +259,7 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& com } return Status::OK(); - } +} } // namespace example } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index ead7169bd63..4464920f17b 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -62,12 +62,16 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { const ServerCallContext& context, std::unique_ptr* result) override; - Status GetFlightInfoTables(const pb::sql::CommandGetTables &command, const ServerCallContext &context, - const FlightDescriptor &descriptor, std::unique_ptr *info) override; + Status GetFlightInfoTables(const pb::sql::CommandGetTables& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) override; - Status DoGetTables(const pb::sql::CommandGetTables &command, const ServerCallContext &context, - std::unique_ptr *result) override; -private: + Status DoGetTables(const pb::sql::CommandGetTables& command, + const ServerCallContext& context, + std::unique_ptr* result) override; + + private: sqlite3* db_; }; diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp index d23171c17c6..0d6417a098f 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp @@ -1,13 +1,27 @@ +// 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 // -// Created by jose on 9/24/21. +// 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 "sqlite_tables_schema_batch_reader.h" +#include "arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h" #include #include #include #include + #include #include "arrow/flight/flight-sql/example/sqlite_statement.h" @@ -17,7 +31,7 @@ namespace flight { namespace sql { std::shared_ptr SqliteTablesWithSchemaBatchReader::schema() const { - return SqlSchema::GetTablesSchemaWithSchema(); + return SqlSchema::GetTablesSchemaWithIncludedSchema(); } Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* batch) { @@ -49,19 +63,19 @@ Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* const std::string& table_name = string_array->GetString(i); std::vector> column_fields; - while(sqlite3_step(schema_statement-> GetSqlite3Stmt()) == SQLITE_ROW) { - const char* string1 = - reinterpret_cast(sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 0)); + while (sqlite3_step(schema_statement->GetSqlite3Stmt()) == SQLITE_ROW) { + const char* string1 = reinterpret_cast( + sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 0)); std::string string = std::string(string1); if (string == table_name) { - const char* column_name = (char*)sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 1); - const char* column_type = (char*) sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 2); + const char* column_name = + (char*)sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 1); + const char* column_type = + (char*)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)); + column_fields.push_back( + arrow::field(column_name, GetArrowType(column_type), nullable == 0, NULL)); } } const arrow::Result>& value = @@ -80,21 +94,22 @@ Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* return Status::OK(); } -std::shared_ptr SqliteTablesWithSchemaBatchReader::GetArrowType(const std::string& sqlite_type) { - if(boost::iequals(sqlite_type, "int") || boost::iequals(sqlite_type, "integer")) { +std::shared_ptr SqliteTablesWithSchemaBatchReader::GetArrowType( + const std::string& sqlite_type) { + if (boost::iequals(sqlite_type, "int") || boost::iequals(sqlite_type, "integer")) { return int64(); - } else if ( boost::iequals(sqlite_type, "REAL")) { + } else if (boost::iequals(sqlite_type, "REAL")) { return float64(); - } else if(boost::iequals(sqlite_type, "BLOB")) { + } else if (boost::iequals(sqlite_type, "BLOB")) { return binary(); } else if (boost::iequals(sqlite_type, "TEXT") || - boost::icontains(sqlite_type, "char") || - boost::icontains(sqlite_type, "varchar")) { + boost::icontains(sqlite_type, "char") || + boost::icontains(sqlite_type, "varchar")) { return utf8(); } else { return null(); } - } +} } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h index 7e985323400..e1c117bb5e4 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h @@ -1,12 +1,28 @@ +// 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 // -// Created by jose on 9/24/21. +// 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. #ifndef ARROW_SQLITE_TABLES_SCHEMA_BATCH_READER_H #define ARROW_SQLITE_TABLES_SCHEMA_BATCH_READER_H #include #include + +#include + #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index 608e984f54c..e9a03649a05 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -260,7 +260,7 @@ std::shared_ptr SqlSchema::GetTablesSchema() { field("table_name", utf8()), field("table_type", utf8())}); } -std::shared_ptr SqlSchema::GetTablesSchemaWithSchema() { +std::shared_ptr SqlSchema::GetTablesSchemaWithIncludedSchema() { return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8()), field("table_name", utf8()), field("table_type", utf8()), field("table_schema", binary())}); diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index bcf4755c61e..1e52dbb3c99 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -267,8 +267,14 @@ class SqlSchema { /// \brief Gets the Schema used on CommandGetSchemas response. static std::shared_ptr GetSchemasSchema(); + + /// \brief Gets the Schema used on CommandGetTables response when included schema + /// flags is set to false. static std::shared_ptr GetTablesSchema(); - static std::shared_ptr GetTablesSchemaWithSchema(); + + /// \brief Gets the Schema used on CommandGetTables response when included schema + /// flags is set to true. + static std::shared_ptr GetTablesSchemaWithIncludedSchema(); }; } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index d384da842d8..378798a8c0c 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -17,7 +17,6 @@ #include #include -#include #include #include #include @@ -41,16 +40,16 @@ ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ } -#define DECLARE_BINARY_ARRAY(ARRAY_NAME, DATA, LENGTH) \ -std::shared_ptr ARRAY_NAME; \ -{ \ -arrow::Binary##Builder builder; \ -auto data = unparen DATA; \ -for (const auto& item : data) { \ -ASSERT_OK(builder.Append(item, LENGTH)); \ -} \ -ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ -} +#define DECLARE_BINARY_ARRAY(ARRAY_NAME, DATA, LENGTH) \ + std::shared_ptr ARRAY_NAME; \ + { \ + arrow::Binary##Builder builder; \ + auto data = unparen DATA; \ + for (const auto& item : data) { \ + ASSERT_OK(builder.Append(item, LENGTH)); \ + } \ + ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ + } using ::testing::_; using ::testing::Ref; @@ -120,7 +119,8 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { TEST(TestFlightSqlServer, TestCommandGetTables) { std::unique_ptr flight_info; std::vector table_types; - ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types, + &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); @@ -128,7 +128,8 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master"})); + DECLARE_ARRAY(catalog_name, String, + ({"sqlite_master", "sqlite_master", "sqlite_master"})); DECLARE_ARRAY(schema_name, String, ({"main", "main", "main"})); DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); @@ -144,7 +145,8 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { std::vector table_types; std::string table_filter_pattern = "int%"; - ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, false, + table_types, &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); @@ -153,7 +155,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { ASSERT_OK(stream->ReadAll(&table)); DECLARE_ARRAY(catalog_name, String, ({"sqlite_master"})); - DECLARE_ARRAY(schema_name, String, ({"main" })); + DECLARE_ARRAY(schema_name, String, ({"main"})); DECLARE_ARRAY(table_name, String, ({"intTable"})); DECLARE_ARRAY(table_type, String, ({"table"})); @@ -168,7 +170,8 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithSchemaFilter) { std::vector table_types; std::string schema_filter_pattern = "main"; - ASSERT_OK(sql_client->GetTables({}, nullptr, &schema_filter_pattern, nullptr, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, nullptr, &schema_filter_pattern, nullptr, false, + table_types, &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); @@ -176,7 +179,8 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithSchemaFilter) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master"})); + DECLARE_ARRAY(catalog_name, String, + ({"sqlite_master", "sqlite_master", "sqlite_master"})); DECLARE_ARRAY(schema_name, String, ({"main", "main", "main"})); DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); @@ -192,7 +196,8 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithCatalogFilter) { std::vector table_types; std::string catalog_filter_pattern = "sqlite_master"; - ASSERT_OK(sql_client->GetTables({}, &catalog_filter_pattern, nullptr, nullptr, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, &catalog_filter_pattern, nullptr, nullptr, false, + table_types, &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); @@ -200,7 +205,8 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithCatalogFilter) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master"})); + DECLARE_ARRAY(catalog_name, String, + ({"sqlite_master", "sqlite_master", "sqlite_master"})); DECLARE_ARRAY(schema_name, String, ({"main", "main", "main"})); DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); @@ -216,7 +222,8 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceSchemaFilter) { std::vector table_types; std::string schema_filter_pattern = "unknown"; - ASSERT_OK(sql_client->GetTables({}, nullptr, &schema_filter_pattern, nullptr, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, nullptr, &schema_filter_pattern, nullptr, false, + table_types, &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); @@ -234,7 +241,8 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceCatalogFilter) { std::vector table_types; std::string catalog_filter_pattern = "unknown"; - ASSERT_OK(sql_client->GetTables({}, &catalog_filter_pattern, nullptr, nullptr, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, &catalog_filter_pattern, nullptr, nullptr, false, + table_types, &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); @@ -249,9 +257,10 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceCatalogFilter) { TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { std::unique_ptr flight_info; - std::vector table_types {"index"}; + std::vector table_types{"index"}; - ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types, + &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); @@ -266,9 +275,10 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { std::unique_ptr flight_info; - std::vector table_types {"table"}; + std::vector table_types{"table"}; - ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types, + &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); @@ -276,7 +286,8 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(catalog_name, String, ({"sqlite_master", "sqlite_master", "sqlite_master"})); + DECLARE_ARRAY(catalog_name, String, + ({"sqlite_master", "sqlite_master", "sqlite_master"})); DECLARE_ARRAY(schema_name, String, ({"main", "main", "main"})); DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); @@ -292,7 +303,8 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { std::vector table_types; std::string table_filter_pattern = "int%"; - ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, true, table_types, &flight_info)); + ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, true, + table_types, &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); @@ -301,25 +313,26 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { ASSERT_OK(stream->ReadAll(&table)); DECLARE_ARRAY(catalog_name, String, ({"sqlite_master"})); - DECLARE_ARRAY(schema_name, String, ({"main" })); + DECLARE_ARRAY(schema_name, String, ({"main"})); DECLARE_ARRAY(table_name, String, ({"intTable"})); DECLARE_ARRAY(table_type, String, ({"table"})); - const std::shared_ptr schema_table = arrow::schema( - {arrow::field("id", int64(), true, NULL), - arrow::field("keyName", utf8(), true, NULL), - arrow::field("value", int64(), true, NULL), - arrow::field("foreignId", int64(), true, NULL) - }); + const std::shared_ptr schema_table = + arrow::schema({arrow::field("id", int64(), true, NULL), + arrow::field("keyName", utf8(), true, NULL), + arrow::field("value", int64(), true, NULL), + arrow::field("foreignId", int64(), true, NULL)}); const arrow::Result>& value = ipc::SerializeSchema(*schema_table); value.ValueOrDie()->data(), value.ValueOrDie()->size(); - DECLARE_BINARY_ARRAY(table_schema, ({value.ValueOrDie()->data()}), value.ValueOrDie()->size()); + DECLARE_BINARY_ARRAY(table_schema, ({value.ValueOrDie()->data()}), + value.ValueOrDie()->size()); - const std::shared_ptr
& expected_table = Table::Make( - SqlSchema::GetTablesSchemaWithSchema(), {catalog_name, schema_name, table_name, table_type, table_schema}); + const std::shared_ptr
& expected_table = + Table::Make(SqlSchema::GetTablesSchemaWithIncludedSchema(), + {catalog_name, schema_name, table_name, table_type, table_schema}); ASSERT_TRUE(expected_table->Equals(*table)); } @@ -356,7 +369,8 @@ TEST(TestFlightSqlServer, TestCommandGetSchemas) { ASSERT_EQ(0, table->num_rows()); } -auto env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); +auto env = + ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); } // namespace sql } // namespace flight From b8d8396a1b2585d4f02895c91a850fe90d559550 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 15:39:52 -0300 Subject: [PATCH 058/237] change constructor initialization --- .../example/sqlite_tables_schema_batch_reader.h | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h index e1c117bb5e4..eb7b4d17bac 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h @@ -35,15 +35,21 @@ class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { std::shared_ptr reader_; sqlite3* db_; - SqliteTablesWithSchemaBatchReader(std::shared_ptr reader, - sqlite3* db_) - : reader_(reader), db_(db_) {} + /// Constructor for SqliteTablesWithSchemaBatchReader class + /// \param reader an shared_ptr from a SqliteStatementBatchReader. + /// \param db_ a pointer to the sqlite3 db. + SqliteTablesWithSchemaBatchReader( + std::shared_ptr p_reader, sqlite3* p_db_) + : reader_(std::move(p_reader)), db_(p_db_) {} std::shared_ptr schema() const override; Status ReadNext(std::shared_ptr* batch) override; - std::shared_ptr GetArrowType(const std::string& sqlite_type); + /// Convert a column type to a ArrowType. + /// \param sqlite_type the sqlite type. + /// \return The equivalent ArrowType. + static std::shared_ptr GetArrowType(const std::string& sqlite_type); }; } // namespace sql } // namespace flight From 0aa1cab6dd1743d59ed5f9252b7fc3cd54649e4e Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 16:43:41 -0300 Subject: [PATCH 059/237] Modify Macro from builder to deal with null values --- .../example/sqlite_statement_batch_reader.cc | 36 +++++++++++-------- 1 file changed, 22 insertions(+), 14 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index c67117d178a..8eda1fc788e 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -22,22 +22,30 @@ #include "arrow/api.h" #include "arrow/flight/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); \ - ARROW_RETURN_NOT_OK( \ - (dynamic_cast(builder)).Append(string, bytes)); \ - break; \ +#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((dynamic_cast(builder)).AppendNull()); \ + break; \ + } \ + ARROW_RETURN_NOT_OK( \ + (dynamic_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); \ - ARROW_RETURN_NOT_OK( \ - (dynamic_cast(builder)).Append((char*)blob, 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((dynamic_cast(builder)).AppendNull()); \ + break; \ + } \ + ARROW_RETURN_NOT_OK( \ + (dynamic_cast(builder)).Append((char*)blob, bytes)); \ + break; \ } #define INT_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ From 574878f2e7cec50658b19fd3668a0755795d8769 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 16:43:57 -0300 Subject: [PATCH 060/237] Set values from catalogs and schema as null --- .../flight-sql/example/sqlite_server.cc | 2 +- .../flight/flight-sql/sql_server_test.cc | 119 +++--------------- 2 files changed, 20 insertions(+), 101 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index a604a400d1b..391d521d10d 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -210,7 +210,7 @@ Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& comma const ServerCallContext& context, std::unique_ptr* result) { std::string table_query( - "SELECT 'sqlite_master' as catalog_name, 'main' as schema_name, name as " + "SELECT null as catalog_name, null as schema_name, name as " "table_name, type as table_type FROM sqlite_master where 1=1"); std::shared_ptr batch_reader; diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 378798a8c0c..9b201a0f1ee 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -41,7 +41,7 @@ } #define DECLARE_BINARY_ARRAY(ARRAY_NAME, DATA, LENGTH) \ - std::shared_ptr ARRAY_NAME; \ + std::shared_ptr ARRAY_NAME; \ { \ arrow::Binary##Builder builder; \ auto data = unparen DATA; \ @@ -51,6 +51,16 @@ ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ } +#define DECLARE_NULL_ARRAY(ARRAY_NAME, TYPE_CLASS, LENGTH) \ + std::shared_ptr ARRAY_NAME; \ + { \ + arrow::TYPE_CLASS##Builder builder; \ + for (int i = 0; i < LENGTH; i++) { \ + ASSERT_OK(builder.AppendNull()); \ + } \ + ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ + } + using ::testing::_; using ::testing::Ref; @@ -128,9 +138,8 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(catalog_name, String, - ({"sqlite_master", "sqlite_master", "sqlite_master"})); - DECLARE_ARRAY(schema_name, String, ({"main", "main", "main"})); + DECLARE_NULL_ARRAY(catalog_name, String,3); + DECLARE_NULL_ARRAY(schema_name, String, 3); DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); @@ -154,8 +163,8 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(catalog_name, String, ({"sqlite_master"})); - DECLARE_ARRAY(schema_name, String, ({"main"})); + DECLARE_NULL_ARRAY(catalog_name, String, 1); + DECLARE_NULL_ARRAY(schema_name, String, 1); DECLARE_ARRAY(table_name, String, ({"intTable"})); DECLARE_ARRAY(table_type, String, ({"table"})); @@ -165,95 +174,6 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { ASSERT_TRUE(expected_table->Equals(*table)); } -TEST(TestFlightSqlServer, TestCommandGetTablesWithSchemaFilter) { - std::unique_ptr flight_info; - std::vector table_types; - - std::string schema_filter_pattern = "main"; - ASSERT_OK(sql_client->GetTables({}, nullptr, &schema_filter_pattern, nullptr, false, - table_types, &flight_info)); - - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); - - std::shared_ptr
table; - ASSERT_OK(stream->ReadAll(&table)); - - DECLARE_ARRAY(catalog_name, String, - ({"sqlite_master", "sqlite_master", "sqlite_master"})); - DECLARE_ARRAY(schema_name, String, ({"main", "main", "main"})); - DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); - DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); - - const std::shared_ptr
& expected_table = Table::Make( - SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); - - ASSERT_TRUE(expected_table->Equals(*table)); -} - -TEST(TestFlightSqlServer, TestCommandGetTablesWithCatalogFilter) { - std::unique_ptr flight_info; - std::vector table_types; - - std::string catalog_filter_pattern = "sqlite_master"; - ASSERT_OK(sql_client->GetTables({}, &catalog_filter_pattern, nullptr, nullptr, false, - table_types, &flight_info)); - - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); - - std::shared_ptr
table; - ASSERT_OK(stream->ReadAll(&table)); - - DECLARE_ARRAY(catalog_name, String, - ({"sqlite_master", "sqlite_master", "sqlite_master"})); - DECLARE_ARRAY(schema_name, String, ({"main", "main", "main"})); - DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); - DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); - - const std::shared_ptr
& expected_table = Table::Make( - SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); - - ASSERT_TRUE(expected_table->Equals(*table)); -} - -TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceSchemaFilter) { - std::unique_ptr flight_info; - std::vector table_types; - - std::string schema_filter_pattern = "unknown"; - ASSERT_OK(sql_client->GetTables({}, nullptr, &schema_filter_pattern, nullptr, false, - table_types, &flight_info)); - - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); - - std::shared_ptr
table; - ASSERT_OK(stream->ReadAll(&table)); - - ASSERT_TRUE(table->schema()->Equals(SqlSchema::GetTablesSchema())); - - ASSERT_EQ(table->num_rows(), 0); -} - -TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceCatalogFilter) { - std::unique_ptr flight_info; - std::vector table_types; - - std::string catalog_filter_pattern = "unknown"; - ASSERT_OK(sql_client->GetTables({}, &catalog_filter_pattern, nullptr, nullptr, false, - table_types, &flight_info)); - - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); - - std::shared_ptr
table; - ASSERT_OK(stream->ReadAll(&table)); - - ASSERT_TRUE(table->schema()->Equals(SqlSchema::GetTablesSchema())); - - ASSERT_EQ(table->num_rows(), 0); -} TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { std::unique_ptr flight_info; @@ -286,9 +206,8 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(catalog_name, String, - ({"sqlite_master", "sqlite_master", "sqlite_master"})); - DECLARE_ARRAY(schema_name, String, ({"main", "main", "main"})); + DECLARE_NULL_ARRAY(catalog_name, String,3); + DECLARE_NULL_ARRAY(schema_name, String, 3); DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); @@ -312,8 +231,8 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(catalog_name, String, ({"sqlite_master"})); - DECLARE_ARRAY(schema_name, String, ({"main"})); + DECLARE_NULL_ARRAY(catalog_name, String, 1); + DECLARE_NULL_ARRAY(schema_name, String, 1); DECLARE_ARRAY(table_name, String, ({"intTable"})); DECLARE_ARRAY(table_type, String, ({"table"})); From 9a838a9dd04279526b5b3d25b683556f0526b6c6 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 16:44:07 -0300 Subject: [PATCH 061/237] Update CMakeLists.txt --- cpp/src/arrow/flight/flight-sql/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 7334445dab2..71f9516987d 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -20,7 +20,7 @@ add_custom_target(arrow_flight_sql) arrow_install_all_headers("arrow/flight/flight-sql") set(FLIGHT_SQL_PROTO_PATH "${ARROW_SOURCE_DIR}/../format") -set(FLIGHT_SQL_PROTO ${ARROW_SOURCE_DIR}/../format/FlightSql.proto) +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") From bd19a660f588e6a4e17907a6ff972e05407aa0cf Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 16:45:24 -0300 Subject: [PATCH 062/237] Fix checkstyle --- cpp/src/arrow/flight/flight-sql/sql_server_test.cc | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 9b201a0f1ee..c7b7793ebe6 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -41,7 +41,7 @@ } #define DECLARE_BINARY_ARRAY(ARRAY_NAME, DATA, LENGTH) \ - std::shared_ptr ARRAY_NAME; \ + std::shared_ptr ARRAY_NAME; \ { \ arrow::Binary##Builder builder; \ auto data = unparen DATA; \ @@ -138,7 +138,7 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_NULL_ARRAY(catalog_name, String,3); + DECLARE_NULL_ARRAY(catalog_name, String, 3); DECLARE_NULL_ARRAY(schema_name, String, 3); DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); @@ -174,7 +174,6 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { ASSERT_TRUE(expected_table->Equals(*table)); } - TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { std::unique_ptr flight_info; std::vector table_types{"index"}; @@ -206,7 +205,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_NULL_ARRAY(catalog_name, String,3); + DECLARE_NULL_ARRAY(catalog_name, String, 3); DECLARE_NULL_ARRAY(schema_name, String, 3); DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); From 4a99053103e53a08c6f8398d3473351ea8f450b2 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 1 Oct 2021 18:14:15 -0300 Subject: [PATCH 063/237] Add method prepareQuery --- .../flight-sql/example/sqlite_server.cc | 100 ++++++++++-------- .../flight/flight-sql/example/sqlite_server.h | 5 + 2 files changed, 58 insertions(+), 47 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 391d521d10d..c83c162382a 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -192,16 +192,14 @@ Status SQLiteFlightSqlServer::GetFlightInfoTables( std::vector endpoints{ FlightEndpoint{{ticketParsed.SerializeAsString()}, {}}}; - if (command.include_schema()) { - ARROW_ASSIGN_OR_RAISE( - auto result, FlightInfo::Make(*SqlSchema::GetTablesSchemaWithIncludedSchema(), - descriptor, endpoints, -1, -1)) - *info = std::unique_ptr(new FlightInfo(result)); - } else { - ARROW_ASSIGN_OR_RAISE(auto result, FlightInfo::Make(*SqlSchema::GetTablesSchema(), - descriptor, endpoints, -1, -1)) - *info = std::unique_ptr(new FlightInfo(result)); - } + bool include_schema = command.include_schema(); + + ARROW_ASSIGN_OR_RAISE( + auto result, + FlightInfo::Make(include_schema ? *SqlSchema::GetTablesSchemaWithIncludedSchema() + : *SqlSchema::GetTablesSchema(), + descriptor, endpoints, -1, -1)) + *info = std::unique_ptr(new FlightInfo(result)); return Status::OK(); } @@ -209,57 +207,65 @@ Status SQLiteFlightSqlServer::GetFlightInfoTables( Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, std::unique_ptr* result) { - std::string 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"); - - std::shared_ptr batch_reader; - - if (command.has_catalog()) { - table_query = table_query + " and catalog_name='" + command.catalog() + "'"; - } - - if (command.has_schema_filter_pattern()) { - table_query = - table_query + " and schema_name LIKE '" + command.schema_filter_pattern() + "'"; - } - - if (command.has_table_name_filter_pattern()) { - table_query = table_query + " and table_name LIKE '" + - command.table_name_filter_pattern() + "'"; - } - - if (command.table_types_size() > 0) { - google::protobuf::RepeatedPtrField types = command.table_types(); - - std::stringstream ss; - int size = types.size(); - for (int i = 0; i < size; i++) { - ss << "'" << types.at(i) << "'"; - if (size - 1 != i) { - ss << ","; - } - } - table_query = table_query + " and table_type IN (" + ss.str() + ")"; - } + std::stringstream table_query = PrepareQuery(command); std::shared_ptr statement; - ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, table_query, &statement)); + ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, table_query.str(), &statement)); std::shared_ptr reader; ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create( statement, SqlSchema::GetTablesSchema(), &reader)); if (command.include_schema()) { - std::shared_ptr pReader = - std::make_shared(reader, db_); - *result = std::unique_ptr(new RecordBatchStream(pReader)); + std::shared_ptr table_schema_reader = + std::make_shared(reader, table_query.str(), db_); + *result = std::unique_ptr(new RecordBatchStream(table_schema_reader)); } else { *result = std::unique_ptr(new RecordBatchStream(reader)); } return Status::OK(); } + + std::stringstream SQLiteFlightSqlServer::PrepareQuery(const pb::sql::CommandGetTables &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"; + + std::shared_ptr batch_reader; + + if (command.has_catalog()) { + table_query << " and catalog_name='" << command.catalog() << "'"; + } + + if (command.has_schema_filter_pattern()) { + table_query << " and schema_name LIKE '" << command.schema_filter_pattern() << "'"; + } + + if (command.has_table_name_filter_pattern()) { + table_query << " and table_name LIKE '" << + command.table_name_filter_pattern() << "'"; + } + + if (command.table_types_size() > 0) { + google::protobuf::RepeatedPtrField types = command.table_types(); + + table_query << " and table_type IN ("; + int size = types.size(); + for (int i = 0; i < size; i++) { + table_query << "'" << types.at(i) << "'"; + if (size - 1 != i) { + table_query << ","; + } + } + + table_query << ")"; + } + + table_query << " order by table_name"; + return table_query; + } } // namespace example } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index 4464920f17b..5e2d77540f7 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -73,6 +73,11 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { private: sqlite3* db_; + + /// \brief Prepare query for execute a CommandGetTables. + /// \param command a CommandGetTables request. + /// \return a string stream with the mounted query. + static std::stringstream PrepareQuery(const pb::sql::CommandGetTables &command) ; }; } // namespace example From abe5af8dd3412531a46aa35fa03d02b78200481f Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 1 Oct 2021 18:15:14 -0300 Subject: [PATCH 064/237] Invert order of vector on tests --- cpp/src/arrow/flight/flight-sql/sql_server_test.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index c7b7793ebe6..3ca21759229 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -140,7 +140,7 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { DECLARE_NULL_ARRAY(catalog_name, String, 3); DECLARE_NULL_ARRAY(schema_name, String, 3); - DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); + DECLARE_ARRAY(table_name, String, ({"foreignTable", "intTable", "sqlite_sequence", })); DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); const std::shared_ptr
& expected_table = Table::Make( @@ -207,7 +207,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { DECLARE_NULL_ARRAY(catalog_name, String, 3); DECLARE_NULL_ARRAY(schema_name, String, 3); - DECLARE_ARRAY(table_name, String, ({"foreignTable", "sqlite_sequence", "intTable"})); + DECLARE_ARRAY(table_name, String, ({"foreignTable", "intTable", "sqlite_sequence", })); DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); const std::shared_ptr
& expected_table = Table::Make( From e5ea5eba96bb9fbce298454441cd2d62cd7a1b7c Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 1 Oct 2021 18:15:41 -0300 Subject: [PATCH 065/237] Pass the query to the batch reader vector for the table with schemas --- .../example/sqlite_tables_schema_batch_reader.cpp | 10 ++++++---- .../example/sqlite_tables_schema_batch_reader.h | 6 ++++-- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp index 0d6417a098f..ca16ebdcc09 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include @@ -35,13 +36,14 @@ std::shared_ptr SqliteTablesWithSchemaBatchReader::schema() const { } Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* batch) { - const std::string schema_query = - "SELECT table_name, name, type, [notnull] FROM pragma_table_info(table_name)" - " JOIN (SELECT tbl_name as table_name FROM sqlite_master)"; + 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_RETURN_NOT_OK( - example::SqliteStatement::Create(db_, schema_query, &schema_statement)); + example::SqliteStatement::Create(db_, schema_query.str(), &schema_statement)); std::shared_ptr first_batch; diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h index eb7b4d17bac..723aa26c029 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h @@ -22,6 +22,7 @@ #include #include +#include #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" @@ -33,14 +34,15 @@ namespace sql { class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { public: std::shared_ptr reader_; + std::string main_query; sqlite3* db_; /// Constructor for SqliteTablesWithSchemaBatchReader class /// \param reader an shared_ptr from a SqliteStatementBatchReader. /// \param db_ a pointer to the sqlite3 db. SqliteTablesWithSchemaBatchReader( - std::shared_ptr p_reader, sqlite3* p_db_) - : reader_(std::move(p_reader)), db_(p_db_) {} + std::shared_ptr p_reader, std::string p_main_query, sqlite3* p_db_) + : reader_(std::move(p_reader)), main_query(std::move(p_main_query)) , db_(p_db_){} std::shared_ptr schema() const override; From 449938b94b5b1d4ba241ec31ad57ee3ab91aee15 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 5 Oct 2021 16:00:16 -0300 Subject: [PATCH 066/237] Remove extra space --- cpp/src/arrow/flight/flight-sql/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 71f9516987d..7334445dab2 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -20,7 +20,7 @@ add_custom_target(arrow_flight_sql) arrow_install_all_headers("arrow/flight/flight-sql") set(FLIGHT_SQL_PROTO_PATH "${ARROW_SOURCE_DIR}/../format") -set(FLIGHT_SQL_PROTO ${ARROW_SOURCE_DIR}/../format/FlightSql.proto) +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") From 5e2071c84e3775f09814548655986456bd257f1a Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 5 Oct 2021 16:01:11 -0300 Subject: [PATCH 067/237] Decouple method PrepareQueryForGetTables from the class --- .../flight-sql/example/sqlite_server.cc | 86 ++++++++++--------- .../flight/flight-sql/example/sqlite_server.h | 5 -- 2 files changed, 44 insertions(+), 47 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index c83c162382a..76785e20b3a 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -204,13 +204,54 @@ Status SQLiteFlightSqlServer::GetFlightInfoTables( return Status::OK(); } + +std::string PrepareQueryForGetTables(const pb::sql::CommandGetTables &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"; + + std::shared_ptr batch_reader; + + if (command.has_catalog()) { + table_query << " and catalog_name='" << command.catalog() << "'"; + } + + if (command.has_schema_filter_pattern()) { + table_query << " and schema_name LIKE '" << command.schema_filter_pattern() << "'"; + } + + if (command.has_table_name_filter_pattern()) { + table_query << " and table_name LIKE '" << + command.table_name_filter_pattern() << "'"; + } + + if (!command.table_types().empty()) { + google::protobuf::RepeatedPtrField types = command.table_types(); + + table_query << " and table_type IN ("; + int size = types.size(); + for (int i = 0; i < size; i++) { + table_query << "'" << types.at(i) << "'"; + if (size - 1 != i) { + table_query << ","; + } + } + + table_query << ")"; + } + + table_query << " order by table_name"; + return table_query.str(); +} + Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, std::unique_ptr* result) { - std::stringstream table_query = PrepareQuery(command); + std::string query = PrepareQueryForGetTables(command); std::shared_ptr statement; - ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, table_query.str(), &statement)); + ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, query, &statement)); std::shared_ptr reader; ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create( @@ -218,7 +259,7 @@ Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& comma if (command.include_schema()) { std::shared_ptr table_schema_reader = - std::make_shared(reader, table_query.str(), db_); + std::make_shared(reader, query, db_); *result = std::unique_ptr(new RecordBatchStream(table_schema_reader)); } else { *result = std::unique_ptr(new RecordBatchStream(reader)); @@ -227,45 +268,6 @@ Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& comma return Status::OK(); } - std::stringstream SQLiteFlightSqlServer::PrepareQuery(const pb::sql::CommandGetTables &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"; - - std::shared_ptr batch_reader; - - if (command.has_catalog()) { - table_query << " and catalog_name='" << command.catalog() << "'"; - } - - if (command.has_schema_filter_pattern()) { - table_query << " and schema_name LIKE '" << command.schema_filter_pattern() << "'"; - } - - if (command.has_table_name_filter_pattern()) { - table_query << " and table_name LIKE '" << - command.table_name_filter_pattern() << "'"; - } - - if (command.table_types_size() > 0) { - google::protobuf::RepeatedPtrField types = command.table_types(); - - table_query << " and table_type IN ("; - int size = types.size(); - for (int i = 0; i < size; i++) { - table_query << "'" << types.at(i) << "'"; - if (size - 1 != i) { - table_query << ","; - } - } - - table_query << ")"; - } - - table_query << " order by table_name"; - return table_query; - } } // namespace example } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index 5e2d77540f7..4464920f17b 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -73,11 +73,6 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { private: sqlite3* db_; - - /// \brief Prepare query for execute a CommandGetTables. - /// \param command a CommandGetTables request. - /// \return a string stream with the mounted query. - static std::stringstream PrepareQuery(const pb::sql::CommandGetTables &command) ; }; } // namespace example From 61af7d444e635855fe0854dfba49d26cf75f38ac Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 5 Oct 2021 16:01:22 -0300 Subject: [PATCH 068/237] Fix import order --- cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 76785e20b3a..529b56c18ca 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -21,9 +21,9 @@ #include "arrow/api.h" #include "arrow/flight/flight-sql/sql_server.h" +#include "arrow/flight/flight-sql/example/sqlite_server.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" -#include "arrow/flight/flight-sql/example/sqlite_server.h" #include "arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h" namespace arrow { From 9f90bdec11c5827d6ac790f5345a2707b5d1b9a5 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 5 Oct 2021 16:01:52 -0300 Subject: [PATCH 069/237] Refactor constructor from the class sqlite_tables_schema_batch_reader --- .../flight-sql/example/sqlite_statement.cc | 2 +- .../sqlite_tables_schema_batch_reader.cpp | 20 +++++++++---------- .../sqlite_tables_schema_batch_reader.h | 9 +++++---- 3 files changed, 16 insertions(+), 15 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index 248a4e8ffda..87f834ea75f 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -46,7 +46,7 @@ Status SqliteStatement::Create(sqlite3* db, const std::string& sql, std::shared_ptr* result) { sqlite3_stmt* stmt; int rc = - sqlite3_prepare_v2(db, sql.c_str(), static_cast(sql.length()), &stmt, NULLPTR); + sqlite3_prepare_v2(db, sql.c_str(), static_cast(sql.size()), &stmt, NULLPTR); if (rc != SQLITE_OK) { sqlite3_finalize(stmt); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp index ca16ebdcc09..79d236962c6 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp @@ -39,7 +39,7 @@ Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* 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"; + "JOIN(" << main_query_ << ") order by table_name"; std::shared_ptr schema_statement; ARROW_RETURN_NOT_OK( @@ -61,19 +61,18 @@ Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* 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); - std::vector> column_fields; while (sqlite3_step(schema_statement->GetSqlite3Stmt()) == SQLITE_ROW) { - const char* string1 = reinterpret_cast( - sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 0)); - std::string string = std::string(string1); - if (string == table_name) { + 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 = - (char*)sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 1); + reinterpret_cast(sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 1)); const char* column_type = - (char*)sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 2); + reinterpret_cast(sqlite3_column_text(schema_statement->GetSqlite3Stmt(), 2)); int nullable = sqlite3_column_int(schema_statement->GetSqlite3Stmt(), 3); column_fields.push_back( @@ -82,6 +81,7 @@ Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* } const arrow::Result>& value = ipc::SerializeSchema(*arrow::schema(column_fields)); + column_fields.clear(); ARROW_RETURN_NOT_OK( schema_builder.Append(value.ValueOrDie()->data(), value.ValueOrDie()->size())); } @@ -105,8 +105,8 @@ std::shared_ptr SqliteTablesWithSchemaBatchReader::GetArrowType( } else if (boost::iequals(sqlite_type, "BLOB")) { return binary(); } else if (boost::iequals(sqlite_type, "TEXT") || - boost::icontains(sqlite_type, "char") || - boost::icontains(sqlite_type, "varchar")) { + boost::istarts_with(sqlite_type, "char") || + boost::istarts_with(sqlite_type, "varchar")) { return utf8(); } else { return null(); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h index 723aa26c029..05fa599e02d 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h @@ -32,17 +32,18 @@ namespace flight { namespace sql { class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { - public: + private: std::shared_ptr reader_; - std::string main_query; + std::string main_query_; sqlite3* db_; +public: /// Constructor for SqliteTablesWithSchemaBatchReader class /// \param reader an shared_ptr from a SqliteStatementBatchReader. /// \param db_ a pointer to the sqlite3 db. SqliteTablesWithSchemaBatchReader( - std::shared_ptr p_reader, std::string p_main_query, sqlite3* p_db_) - : reader_(std::move(p_reader)), main_query(std::move(p_main_query)) , db_(p_db_){} + std::shared_ptr reader, std::string& main_query, sqlite3* db) + : reader_(std::move(reader)), main_query_(main_query) , db_(db){} std::shared_ptr schema() const override; From d962a33462b2061e9926a540f016d761247fbec5 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 5 Oct 2021 16:04:37 -0300 Subject: [PATCH 070/237] Fix checkstyle --- .../flight-sql/example/sqlite_server.cc | 6 ++++-- .../sqlite_tables_schema_batch_reader.cpp | 19 +++++++++++-------- .../sqlite_tables_schema_batch_reader.h | 9 +++++---- 3 files changed, 20 insertions(+), 14 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 529b56c18ca..757e5e4610b 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -260,9 +260,11 @@ Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& comma if (command.include_schema()) { std::shared_ptr table_schema_reader = std::make_shared(reader, query, db_); - *result = std::unique_ptr(new RecordBatchStream(table_schema_reader)); + *result = std::unique_ptr( + new RecordBatchStream(table_schema_reader)); } else { - *result = std::unique_ptr(new RecordBatchStream(reader)); + *result = std::unique_ptr( + new RecordBatchStream(reader)); } return Status::OK(); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp index 79d236962c6..f1bec9b98aa 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp @@ -21,9 +21,9 @@ #include #include #include -#include #include +#include #include "arrow/flight/flight-sql/example/sqlite_statement.h" @@ -38,8 +38,9 @@ std::shared_ptr SqliteTablesWithSchemaBatchReader::schema() const { 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"; + 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_RETURN_NOT_OK( @@ -69,14 +70,16 @@ Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* 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)); + 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)); + arrow::field(column_name, + GetArrowType(column_type), + nullable == 0, NULL)); } } const arrow::Result>& value = diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h index 05fa599e02d..a43b491c57b 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h @@ -21,8 +21,8 @@ #include #include -#include #include +#include #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" @@ -36,14 +36,15 @@ class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { std::shared_ptr reader_; std::string main_query_; sqlite3* db_; -public: + public: /// Constructor for SqliteTablesWithSchemaBatchReader class /// \param reader an shared_ptr from a SqliteStatementBatchReader. /// \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_(main_query) , db_(db){} + std::shared_ptr reader, + std::string& main_query, sqlite3* db) + : reader_(std::move(reader)), main_query_(main_query), db_(db) {} std::shared_ptr schema() const override; From 05d6280326872b3f7f75bc51b8e8f91ef49979c1 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 11:59:30 -0300 Subject: [PATCH 071/237] Move anonymous function to the top of file --- .../flight-sql/example/sqlite_server.cc | 80 +++++++++---------- 1 file changed, 39 insertions(+), 41 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 757e5e4610b..1cfe006b1a9 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -31,6 +31,45 @@ namespace flight { namespace sql { namespace example { +std::string PrepareQueryForGetTables(const pb::sql::CommandGetTables& 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"; + + std::shared_ptr batch_reader; + + if (command.has_catalog()) { + table_query << " and catalog_name='" << command.catalog() << "'"; + } + + if (command.has_schema_filter_pattern()) { + table_query << " and schema_name LIKE '" << command.schema_filter_pattern() << "'"; + } + + if (command.has_table_name_filter_pattern()) { + table_query << " and table_name LIKE '" << command.table_name_filter_pattern() << "'"; + } + + if (!command.table_types().empty()) { + google::protobuf::RepeatedPtrField types = command.table_types(); + + table_query << " and table_type IN ("; + int size = types.size(); + for (int i = 0; i < size; i++) { + table_query << "'" << types.at(i) << "'"; + if (size - 1 != i) { + table_query << ","; + } + } + + table_query << ")"; + } + + table_query << " order by table_name"; + return table_query.str(); +} + SQLiteFlightSqlServer::SQLiteFlightSqlServer() { db_ = NULLPTR; if (sqlite3_open(":memory:", &db_)) { @@ -204,47 +243,6 @@ Status SQLiteFlightSqlServer::GetFlightInfoTables( return Status::OK(); } - -std::string PrepareQueryForGetTables(const pb::sql::CommandGetTables &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"; - - std::shared_ptr batch_reader; - - if (command.has_catalog()) { - table_query << " and catalog_name='" << command.catalog() << "'"; - } - - if (command.has_schema_filter_pattern()) { - table_query << " and schema_name LIKE '" << command.schema_filter_pattern() << "'"; - } - - if (command.has_table_name_filter_pattern()) { - table_query << " and table_name LIKE '" << - command.table_name_filter_pattern() << "'"; - } - - if (!command.table_types().empty()) { - google::protobuf::RepeatedPtrField types = command.table_types(); - - table_query << " and table_type IN ("; - int size = types.size(); - for (int i = 0; i < size; i++) { - table_query << "'" << types.at(i) << "'"; - if (size - 1 != i) { - table_query << ","; - } - } - - table_query << ")"; - } - - table_query << " order by table_name"; - return table_query.str(); -} - Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, std::unique_ptr* result) { From 4f0807a7eb8d99e363ce07e33364acc164621a89 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 12:00:03 -0300 Subject: [PATCH 072/237] Remove status from sqlite use --- .../example/sqlite_statement_batch_reader.cc | 26 ++++++++++++++----- .../example/sqlite_statement_batch_reader.h | 3 +-- 2 files changed, 21 insertions(+), 8 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index 8eda1fc788e..fcc5dfa512a 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -71,9 +71,9 @@ namespace example { std::shared_ptr SqliteStatementBatchReader::schema() const { return schema_; } -SqliteStatementBatchReader::SqliteStatementBatchReader( - std::shared_ptr statement, std::shared_ptr schema, int rc) - : statement_(std::move(statement)), schema_(std::move(schema)), rc_(rc) {} +SqliteStatementBatchReader::SqliteStatementBatchReader(std::shared_ptr statement, + std::shared_ptr schema) + : statement_(std::move(statement)), schema_(std::move(schema)) {} Status SqliteStatementBatchReader::Create( const std::shared_ptr& statement_, @@ -84,7 +84,16 @@ Status SqliteStatementBatchReader::Create( std::shared_ptr schema; ARROW_RETURN_NOT_OK(statement_->GetSchema(&schema)); - result->reset(new SqliteStatementBatchReader(statement_, schema, rc)); + result->reset(new SqliteStatementBatchReader(statement_, schema)); + + return Status::OK(); +} + +Status SqliteStatementBatchReader::Create( + const std::shared_ptr &statement_, + const std::shared_ptr &schema, + std::shared_ptr *result) { + result->reset(new SqliteStatementBatchReader(statement_, schema)); return Status::OK(); } @@ -102,8 +111,13 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr* out) { ARROW_RETURN_NOT_OK(MakeBuilder(default_memory_pool(), field_type, &builders[i])); } + sqlite3_reset(stmt_); + + int rc; + ARROW_RETURN_NOT_OK(statement_->Step(&rc)); + int rows = 0; - while (rows < MAX_BATCH_SIZE && rc_ == SQLITE_ROW) { + while (rows < MAX_BATCH_SIZE && rc == SQLITE_ROW) { rows++; for (int i = 0; i < num_fields; i++) { const std::shared_ptr& field = schema_->field(i); @@ -132,7 +146,7 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr* out) { } } - ARROW_RETURN_NOT_OK(statement_->Step(&rc_)); + ARROW_RETURN_NOT_OK(statement_->Step(&rc)); } if (rows > 0) { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h index 9b221f8ecb3..4a3c982ddc9 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h @@ -43,10 +43,9 @@ class SqliteStatementBatchReader : public RecordBatchReader { private: std::shared_ptr statement_; std::shared_ptr schema_; - int rc_; SqliteStatementBatchReader(std::shared_ptr statement, - std::shared_ptr schema, int rc); + std::shared_ptr schema); }; } // namespace example From 6bf3fdecf5f55e0715a354c09b89ec5407e14624 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 12:00:26 -0300 Subject: [PATCH 073/237] change parameter from string to char* on GetArrowType method --- .../flight-sql/example/sqlite_tables_schema_batch_reader.cpp | 2 +- .../flight-sql/example/sqlite_tables_schema_batch_reader.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp index f1bec9b98aa..459fe34f2fb 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp @@ -100,7 +100,7 @@ Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* } std::shared_ptr SqliteTablesWithSchemaBatchReader::GetArrowType( - const std::string& sqlite_type) { + const char* sqlite_type) { if (boost::iequals(sqlite_type, "int") || boost::iequals(sqlite_type, "integer")) { return int64(); } else if (boost::iequals(sqlite_type, "REAL")) { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h index a43b491c57b..d44f7d53e58 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h @@ -53,7 +53,7 @@ class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { /// Convert a column type to a ArrowType. /// \param sqlite_type the sqlite type. /// \return The equivalent ArrowType. - static std::shared_ptr GetArrowType(const std::string& sqlite_type); + static std::shared_ptr GetArrowType(const char* sqlite_type); }; } // namespace sql } // namespace flight From ec0319c26d2c44e7f30855066e3212064e7f4d03 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 13:25:19 -0300 Subject: [PATCH 074/237] Fix checkstyle --- .../flight-sql/example/sqlite_statement_batch_reader.cc | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index fcc5dfa512a..42c8074235f 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -71,9 +71,10 @@ namespace example { 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)) {} +SqliteStatementBatchReader::SqliteStatementBatchReader( + std::shared_ptr statement, + std::shared_ptr schema) + : statement_(std::move(statement)), schema_(std::move(schema)) {} Status SqliteStatementBatchReader::Create( const std::shared_ptr& statement_, From d10d1916c3cf7777029b4803f2d86045799a6da6 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 14:15:19 -0300 Subject: [PATCH 075/237] Add a method to reset the statement to its original condition --- .../arrow/flight/flight-sql/example/sqlite_statement.cc | 9 +++++++++ .../arrow/flight/flight-sql/example/sqlite_statement.h | 5 +++++ 2 files changed, 14 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index 87f834ea75f..b4fb6556947 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -83,6 +83,15 @@ Status SqliteStatement::Step(int* rc) { return Status::OK(); } +Status SqliteStatement::Reset(int *rc) { + *rc = sqlite3_reset(stmt_); + if (*rc == SQLITE_ERROR) { + return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db_)); + } + + return Status::OK(); +} + sqlite3_stmt* SqliteStatement::GetSqlite3Stmt() { return stmt_; } } // namespace example diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h index 7d32de342f8..5c573f2df9d 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h @@ -48,6 +48,11 @@ class SqliteStatement { /// \return Status. Status Step(int* rc); + /// \brief Reset the state of the sqlite3_stmt. + /// \param[out] rc The resulting return code from SQLite. + /// \return Status. + Status Reset(int* rc); + /// \brief Returns the underlying sqlite3_stmt. sqlite3_stmt* GetSqlite3Stmt(); From acfaec0df1c08c32907cb05093fd2b23ddf2dbcb Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 14:15:41 -0300 Subject: [PATCH 076/237] Add a variable to control the execution flow --- .../example/sqlite_statement_batch_reader.cc | 12 ++++++++---- .../example/sqlite_statement_batch_reader.h | 2 ++ 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index 42c8074235f..825fbdfd3a1 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -74,7 +74,9 @@ std::shared_ptr SqliteStatementBatchReader::schema() const { return sche SqliteStatementBatchReader::SqliteStatementBatchReader( std::shared_ptr statement, std::shared_ptr schema) - : statement_(std::move(statement)), schema_(std::move(schema)) {} + : statement_(std::move(statement)), schema_(std::move(schema)) { + rc = SQLITE_OK; +} Status SqliteStatementBatchReader::Create( const std::shared_ptr& statement_, @@ -112,10 +114,12 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr* out) { ARROW_RETURN_NOT_OK(MakeBuilder(default_memory_pool(), field_type, &builders[i])); } - sqlite3_reset(stmt_); + if (!already_executed) { + ARROW_RETURN_NOT_OK(statement_->Reset(&rc)); + ARROW_RETURN_NOT_OK(statement_->Step(&rc)); + already_executed = true; + } - int rc; - ARROW_RETURN_NOT_OK(statement_->Step(&rc)); int rows = 0; while (rows < MAX_BATCH_SIZE && rc == SQLITE_ROW) { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h index 4a3c982ddc9..3be69b8f94e 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h @@ -43,6 +43,8 @@ class SqliteStatementBatchReader : public RecordBatchReader { private: std::shared_ptr statement_; std::shared_ptr schema_; + int rc; + bool already_executed; SqliteStatementBatchReader(std::shared_ptr statement, std::shared_ptr schema); From 95e7507622be9037b819a17ed7f1163dad1893a1 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 16:00:20 -0300 Subject: [PATCH 077/237] Fix missing files from rebase --- .../flight-sql/example/sqlite_statement_batch_reader.h | 9 +++++++++ cpp/src/arrow/flight/flight-sql/sql_server_test.cc | 1 - 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h index 3be69b8f94e..5c5b3c84096 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h @@ -36,6 +36,15 @@ class SqliteStatementBatchReader : public RecordBatchReader { static Status Create(const std::shared_ptr& statement, std::shared_ptr* result); + /// \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. + /// \param[out] result The resulting RecordBatchReader. + /// \return Status. + static Status Create(const std::shared_ptr &statement, + const std::shared_ptr &schema, + std::shared_ptr *result); + std::shared_ptr schema() const override; Status ReadNext(std::shared_ptr* out) override; diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 3ca21759229..e2fce861d7a 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -25,7 +25,6 @@ #include #include #include -#include #include #define unparen(...) __VA_ARGS__ From 250a3c4b4a3db0967c17bb8384c8e3ad7976d598 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 16:11:18 -0300 Subject: [PATCH 078/237] Add missing include --- .../flight/flight-sql/example/sqlite_statement_batch_reader.h | 1 + 1 file changed, 1 insertion(+) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h index 5c5b3c84096..a92281ead06 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h @@ -18,6 +18,7 @@ #pragma once #include +#include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/api.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" From b7089548f4219cbb8e994d24da427a0bd060aa00 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 13:32:12 -0300 Subject: [PATCH 079/237] Remove unused RecordBatchReader --- cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc | 2 -- 1 file changed, 2 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 1cfe006b1a9..d78503e2cd8 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -37,8 +37,6 @@ std::string PrepareQueryForGetTables(const pb::sql::CommandGetTables& command) { 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"; - std::shared_ptr batch_reader; - if (command.has_catalog()) { table_query << " and catalog_name='" << command.catalog() << "'"; } From 6c52c281353322d9416e10d2d393bbfe46ffbcfd Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 13:35:22 -0300 Subject: [PATCH 080/237] Passes the rc variable as reference to Reset method --- cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc | 6 +++--- cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h | 2 +- .../flight-sql/example/sqlite_statement_batch_reader.cc | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index b4fb6556947..459389c0f2b 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -83,9 +83,9 @@ Status SqliteStatement::Step(int* rc) { return Status::OK(); } -Status SqliteStatement::Reset(int *rc) { - *rc = sqlite3_reset(stmt_); - if (*rc == SQLITE_ERROR) { +Status SqliteStatement::Reset(int& rc) { + rc = sqlite3_reset(stmt_); + if (rc == SQLITE_ERROR) { return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db_)); } diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h index 5c573f2df9d..311f279f0ad 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h @@ -51,7 +51,7 @@ class SqliteStatement { /// \brief Reset the state of the sqlite3_stmt. /// \param[out] rc The resulting return code from SQLite. /// \return Status. - Status Reset(int* rc); + Status Reset(int& rc); /// \brief Returns the underlying sqlite3_stmt. sqlite3_stmt* GetSqlite3Stmt(); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index 825fbdfd3a1..e638683114d 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -115,7 +115,7 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr* out) { } if (!already_executed) { - ARROW_RETURN_NOT_OK(statement_->Reset(&rc)); + ARROW_RETURN_NOT_OK(statement_->Reset(rc)); ARROW_RETURN_NOT_OK(statement_->Step(&rc)); already_executed = true; } From a7ff4573a72ffed2eb401ae291b29dc0b3477b12 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 13:39:09 -0300 Subject: [PATCH 081/237] Add missing return at docs from SqlSchema methods --- cpp/src/arrow/flight/flight-sql/sql_server.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 1e52dbb3c99..e5b2a521597 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -263,17 +263,21 @@ class FlightSqlServerBase : public FlightServerBase { class SqlSchema { public: /// \brief Gets the Schema used on CommandGetCatalogs response. + /// \return The default schema template. static std::shared_ptr GetCatalogsSchema(); /// \brief Gets the Schema used on CommandGetSchemas response. + /// \return The default schema template. static std::shared_ptr GetSchemasSchema(); /// \brief Gets the Schema used on CommandGetTables response when included schema /// flags is set to false. + /// \return The default schema template. static std::shared_ptr GetTablesSchema(); /// \brief Gets the Schema used on CommandGetTables response when included schema /// flags is set to true. + /// \return The default schema template. static std::shared_ptr GetTablesSchemaWithIncludedSchema(); }; } // namespace sql From 1036ec8c11f1a7992f7a435762702dbc199152c7 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 13:39:21 -0300 Subject: [PATCH 082/237] Remove extra line --- .../flight/flight-sql/example/sqlite_statement_batch_reader.cc | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index e638683114d..6003c1b9870 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -120,7 +120,6 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr* out) { already_executed = true; } - int rows = 0; while (rows < MAX_BATCH_SIZE && rc == SQLITE_ROW) { rows++; From 2a221a6a957c479253fa820c792fec6e573df2c6 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 5 Oct 2021 13:14:55 -0300 Subject: [PATCH 083/237] Implement CommandGetSchemas --- .../flight-sql/example/sqlite_server.cc | 28 +++++++++++++++++++ cpp/src/arrow/flight/flight-sql/sql_server.h | 1 + 2 files changed, 29 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index d78503e2cd8..c5ea9622934 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -16,6 +16,7 @@ // under the License. #include +#include #include @@ -219,6 +220,33 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& com return Status::OK(); } +Status SQLiteFlightSqlServer::GetFlightInfoSchemas( + const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return GetFlightInfoForCommand(descriptor, info, command, + SqlSchema::GetSchemasSchema()); +} + +Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& command, + const ServerCallContext& context, + std::unique_ptr* result) { + // As SQLite doesn't support schemas, this will return an empty record batch. + + const std::shared_ptr& schema = SqlSchema::GetSchemasSchema(); + + 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})); + *result = std::unique_ptr(new RecordBatchStream(reader)); + return Status::OK(); +} + Status SQLiteFlightSqlServer::GetFlightInfoTables( const pb::sql::CommandGetTables& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index e5b2a521597..8ff0bbe0a5d 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -262,6 +262,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \brief Auxiliary class containing all Schemas used on Flight SQL. class SqlSchema { public: + /// \brief Gets the Schema used on CommandGetCatalogs response. /// \return The default schema template. static std::shared_ptr GetCatalogsSchema(); From 70800d0a19d046aefd6b4ced477a9a339846e81d Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 28 Sep 2021 16:09:27 -0300 Subject: [PATCH 084/237] Add the schemas to be used by the GetTabkes --- cpp/src/arrow/flight/flight-sql/sql_server.cpp | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index e9a03649a05..4029d16408c 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -266,6 +266,17 @@ std::shared_ptr SqlSchema::GetTablesSchemaWithIncludedSchema() { field("table_schema", binary())}); } +std::shared_ptr SqlSchema::GetTablesSchema() { + return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8()), + field("table_name", utf8()), field("table_type", utf8())}); +} + +std::shared_ptr SqlSchema::GetTablesSchemaWithSchema() { + return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8()), + field("table_name", utf8()), field("table_type", utf8()), + field("table_schema", binary())}); +} + } // namespace sql } // namespace flight } // namespace arrow From 3066113d217e508d409c27c8eb626682057cd629 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 15:39:18 -0300 Subject: [PATCH 085/237] Fix checkstyle --- cpp/src/arrow/flight/flight-sql/sql_server.cpp | 2 +- cpp/src/arrow/flight/flight-sql/sql_server.h | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index 4029d16408c..667f18b4c81 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -271,7 +271,7 @@ std::shared_ptr SqlSchema::GetTablesSchema() { field("table_name", utf8()), field("table_type", utf8())}); } -std::shared_ptr SqlSchema::GetTablesSchemaWithSchema() { +std::shared_ptr SqlSchema::GetTablesSchemaWithIncludedSchema() { return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8()), field("table_name", utf8()), field("table_type", utf8()), field("table_schema", binary())}); diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 8ff0bbe0a5d..e5b2a521597 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -262,7 +262,6 @@ class FlightSqlServerBase : public FlightServerBase { /// \brief Auxiliary class containing all Schemas used on Flight SQL. class SqlSchema { public: - /// \brief Gets the Schema used on CommandGetCatalogs response. /// \return The default schema template. static std::shared_ptr GetCatalogsSchema(); From 3b08157e1ab88515e4f67c966868ccd31a4ac033 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 29 Sep 2021 17:16:52 -0300 Subject: [PATCH 086/237] Implement CommandGetSchemas --- cpp/src/arrow/flight/protocol_internal.cc | 26 --------------------- cpp/src/arrow/flight/protocol_internal.h | 28 ----------------------- 2 files changed, 54 deletions(-) delete mode 100644 cpp/src/arrow/flight/protocol_internal.cc delete mode 100644 cpp/src/arrow/flight/protocol_internal.h diff --git a/cpp/src/arrow/flight/protocol_internal.cc b/cpp/src/arrow/flight/protocol_internal.cc deleted file mode 100644 index 9f815398e48..00000000000 --- a/cpp/src/arrow/flight/protocol_internal.cc +++ /dev/null @@ -1,26 +0,0 @@ -// 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 - -#include "arrow/flight/protocol_internal.h" - -// NOTE(wesm): Including .cc files in another .cc file would ordinarily be a -// no-no. We have customized the serialization path for FlightData, which is -// currently only possible through some pre-processor commands that need to be -// included before either of these files is compiled. Because we don't want to -// edit the generated C++ files, we include them here and do our gRPC -// customizations in protocol-internal.h -#include "arrow/flight/Flight.grpc.pb.cc" // NOLINT -#include "arrow/flight/Flight.pb.cc" // NOLINT diff --git a/cpp/src/arrow/flight/protocol_internal.h b/cpp/src/arrow/flight/protocol_internal.h deleted file mode 100644 index 98bf9238809..00000000000 --- a/cpp/src/arrow/flight/protocol_internal.h +++ /dev/null @@ -1,28 +0,0 @@ -// 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 - -#pragma once - -// This addresses platform-specific defines, e.g. on Windows -#include "arrow/flight/platform.h" // IWYU pragma: keep - -// This header holds the Flight protobuf definitions. - -// Need to include this first to get our gRPC customizations -#include "arrow/flight/customize_protobuf.h" // IWYU pragma: export - -#include "arrow/flight/Flight.grpc.pb.h" // IWYU pragma: export -#include "arrow/flight/Flight.pb.h" // IWYU pragma: export From 83b1c169d6604b346d373509e301cb10f56aec49 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 5 Oct 2021 22:16:37 -0300 Subject: [PATCH 087/237] Add PreparedStatementClass to sql client --- cpp/src/arrow/flight/flight-sql/client.h | 48 +++++++++++++++++++++++- 1 file changed, 46 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 45bfaae1136..8d38d237e12 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -22,12 +22,46 @@ #include #include #include +#include +#include + + +namespace pb = arrow::flight::protocol; namespace arrow { namespace flight { namespace sql { namespace internal { +template +class ARROW_EXPORT PreparedStatementT { + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + bool is_closed; + T* client; + +public: + /// \brief Constructor for the PreparedStatement class. + /// \param[in] client A raw pointer to FlightClient. + /// \param[in] query The query that will be executed. + PreparedStatementT(T* client, const std::string& query); + + /// \brief Executes the prepared statement query on the server. + /// \param[in] call_options RPC-layer hints for this call. + /// \param[out] info A FlightInfo object representing the stream(s) to fetch. + /// \return Status. + Status Execute(const FlightCallOptions& call_options, + std::unique_ptr *info); + + /// \brief Closes the prepared statement. + /// \param[in] options RPC-layer hints for this call. + /// \return Status. + Status Close(const FlightCallOptions& options); + + /// \brief Checks if the prepared statement is closed. + /// \return The state of the prepared statement. + bool IsClosed() const; +}; + /// \brief Flight client with Flight SQL semantics. template class FlightSqlClientT { @@ -137,13 +171,23 @@ class FlightSqlClientT { Status GetTableTypes(const FlightCallOptions& options, std::unique_ptr* flight_info) const; - private: + /// \brief Create a prepared statement object. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] query The query that will be executed. + /// \param[out] prepared_statement The created prepared statement. + /// \return Status. + Status Prepare(const FlightCallOptions& options, const std::string& query, + std::shared_ptr>* prepared_statement); + +private: std::unique_ptr client; }; } // namespace internal -typedef internal::FlightSqlClientT FlightSqlClient; +using FlightSqlClient = internal::FlightSqlClientT<>; +using PreparedStatement = internal::PreparedStatementT<>; + } // namespace sql } // namespace flight From ee1ed1930d8f3374760388f7ac394f6197174629 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 5 Oct 2021 22:18:12 -0300 Subject: [PATCH 088/237] implemenet methods PreparedStatemenetClass in sql_client --- cpp/src/arrow/flight/flight-sql/client_impl.h | 101 ++++++++++++++++++ 1 file changed, 101 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 16f2d040f72..cc4ba8fc880 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -21,6 +21,7 @@ #include #include + namespace pb = arrow::flight::protocol; namespace arrow { @@ -222,6 +223,106 @@ Status FlightSqlClientT::DoGet(const FlightCallOptions& options, const Ticket return client->DoGet(options, ticket, stream); } +template +Status FlightSqlClientT::Prepare( + const FlightCallOptions& options, const std::string& query, + std::shared_ptr>* prepared_statement) { + *prepared_statement = std::make_shared>(client.get(), query); + + return Status::OK(); +} + +template +PreparedStatementT::PreparedStatementT(T* client_, const std::string& query) + : client(client_) { + google::protobuf::Any command; + pb::sql::ActionCreatePreparedStatementRequest request; + request.set_query(query); + command.PackFrom(request); + + Action action; + action.type = "CreatePreparedStatement"; + action.body = Buffer::FromString(command.SerializeAsString()); + + std::unique_ptr results; + + client->DoAction({}, action, &results); + + std::unique_ptr result; + results->Next(&result); + + google::protobuf::Any prepared_result; + + std::shared_ptr message = result->body; + + prepared_result.ParseFromArray(message->data(), static_cast(message->size())); + + prepared_result.UnpackTo(&prepared_statement_result); + + is_closed = false; +} + +template +Status PreparedStatementT::Execute(const FlightCallOptions& call_options, + std::unique_ptr* info) { + //TODO Treat this error better + if (is_closed) { + std::cout << "Statement is closed" << std::endl; + } + + pb::sql::CommandPreparedStatementQuery prepared_statement_query; + + prepared_statement_query.set_prepared_statement_handle( + prepared_statement_result.prepared_statement_handle()); + + google::protobuf::Any any; + any.PackFrom(prepared_statement_query); + + const std::string& string = any.SerializeAsString(); + const FlightDescriptor& descriptor = FlightDescriptor::Command(string); + + ARROW_RETURN_NOT_OK(client->GetFlightInfo({}, descriptor, info)); + + return Status::OK(); +} + +template +bool PreparedStatementT::IsClosed() const { + return is_closed; +} + +template +Status PreparedStatementT::Close(const FlightCallOptions& options) { + if (is_closed) { + return Status::OK(); + } + + ActionType actionType; + actionType.type = "ClosePreparedStatement"; + actionType.description = + "Closes a reusable prepared statement resource on the server. " + "Request Message: ActionClosePreparedStatementRequest Response Message: N/A;"; + + Action action; + action.type = actionType.type; + google::protobuf::Any command; + pb::sql::ActionClosePreparedStatementRequest request; + request.set_prepared_statement_handle( + prepared_statement_result.prepared_statement_handle()); + + command.PackFrom(request); + + action.body = Buffer::FromString(command.SerializeAsString()); + + std::unique_ptr results; + + ARROW_RETURN_NOT_OK(client->DoAction({}, action, &results)); + + is_closed = true; + + return Status::OK(); +} + } // namespace internal } // namespace sql } // namespace flight From ea75e69ba05b25b5aaa9752a91cd046edd1135f7 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 5 Oct 2021 22:18:31 -0300 Subject: [PATCH 089/237] Create a mocked sql_client test for the preparedStatement --- .../arrow/flight/flight-sql/client_test.cc | 42 ++++++++++++++++++- 1 file changed, 41 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 105411953b2..d72dc168d18 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -43,6 +43,9 @@ class FlightClientMock { 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 FlightMetadataReaderMock : public FlightMetadataReader { @@ -232,6 +235,44 @@ TEST(TestFlightSqlClient, TestExecute) { (void)sqlClient.Execute(call_options, query, &flight_info); } +TEST(TestFlightSqlClient, TestPreparedStatementExecute) { + auto *client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sqlClient(client_mock_ptr); + FlightCallOptions call_options; + + const std::string query = "query"; + + ON_CALL(*client_mock, 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(); + }); + + std::unique_ptr flight_info; + EXPECT_CALL(*client_mock, + DoAction(_, _, _)); + + std::shared_ptr> preparedStatement; + sqlClient.Prepare(query, &preparedStatement); + + EXPECT_CALL(*client_mock, + GetFlightInfo(_, _, &flight_info)); + + (void) preparedStatement->Execute(<#initializer#>, &flight_info); +} + TEST(TestFlightSqlClient, TestExecuteUpdate) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); @@ -278,7 +319,6 @@ TEST(TestFlightSqlClient, TestExecuteUpdate) { ASSERT_EQ(num_rows, 100); } - } // namespace sql } // namespace flight } // namespace arrow From fe96fe096dbba6088808dee434f262aeda5d304b Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 5 Oct 2021 22:31:09 -0300 Subject: [PATCH 090/237] Refactor the creation of the PreparedStatement on sql_client --- cpp/src/arrow/flight/flight-sql/client.h | 2 +- cpp/src/arrow/flight/flight-sql/client_impl.h | 25 +++++++++++-------- .../arrow/flight/flight-sql/client_test.cc | 4 +-- 3 files changed, 17 insertions(+), 14 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 8d38d237e12..0b6b6cc48d8 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -43,7 +43,7 @@ class ARROW_EXPORT PreparedStatementT { /// \brief Constructor for the PreparedStatement class. /// \param[in] client A raw pointer to FlightClient. /// \param[in] query The query that will be executed. - PreparedStatementT(T* client, const std::string& query); + PreparedStatementT(T* client, const std::string& query, std::shared_ptr result_buffer); /// \brief Executes the prepared statement query on the server. /// \param[in] call_options RPC-layer hints for this call. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index cc4ba8fc880..5524571d36e 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -21,6 +21,8 @@ #include #include +#include + namespace pb = arrow::flight::protocol; @@ -227,14 +229,6 @@ template Status FlightSqlClientT::Prepare( const FlightCallOptions& options, const std::string& query, std::shared_ptr>* prepared_statement) { - *prepared_statement = std::make_shared>(client.get(), query); - - return Status::OK(); -} - -template -PreparedStatementT::PreparedStatementT(T* client_, const std::string& query) - : client(client_) { google::protobuf::Any command; pb::sql::ActionCreatePreparedStatementRequest request; request.set_query(query); @@ -246,14 +240,23 @@ PreparedStatementT::PreparedStatementT(T* client_, const std::string& query) std::unique_ptr results; - client->DoAction({}, action, &results); + ARROW_RETURN_NOT_OK(client->DoAction({}, action, &results)); std::unique_ptr result; - results->Next(&result); + ARROW_RETURN_NOT_OK(results->Next(&result)); + + prepared_statement->reset(new PreparedStatementT(client.get(), query, result->body)); + return Status::OK(); +} + +template +PreparedStatementT::PreparedStatementT(T* client_, const std::string& query, + std::shared_ptr result_buffer) + : client(client_) { google::protobuf::Any prepared_result; - std::shared_ptr message = result->body; + std::shared_ptr message = std::move(result_buffer); prepared_result.ParseFromArray(message->data(), static_cast(message->size())); diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index d72dc168d18..52fdfe80770 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -265,12 +265,12 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecute) { DoAction(_, _, _)); std::shared_ptr> preparedStatement; - sqlClient.Prepare(query, &preparedStatement); + sqlClient.Prepare({}, query, &preparedStatement); EXPECT_CALL(*client_mock, GetFlightInfo(_, _, &flight_info)); - (void) preparedStatement->Execute(<#initializer#>, &flight_info); + (void) preparedStatement->Execute({}, &flight_info); } TEST(TestFlightSqlClient, TestExecuteUpdate) { From a5c362c2f59929f3fe4077c87f882a3f1b09ebca Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 5 Oct 2021 22:31:36 -0300 Subject: [PATCH 091/237] Add a branch on test_app to execute a query with PreparedStatement mode --- cpp/src/arrow/flight/flight-sql/test_app.cc | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index 29ba11020a3..ebcf32d792a 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -129,6 +129,13 @@ Status RunMain() { ARROW_RETURN_NOT_OK(sqlClient.Execute(call_options, fLS::FLAGS_query, &info)); } else if (fLS::FLAGS_command == "GetCatalogs") { ARROW_RETURN_NOT_OK(sqlClient.GetCatalogs(call_options, &info)); + } else if (fLS::FLAGS_command == "Prepared") { + std::shared_ptr prepared_statement; + + ARROW_RETURN_NOT_OK(sqlClient.Prepare({}, fLS::FLAGS_query, &prepared_statement)); + ARROW_RETURN_NOT_OK(prepared_statement->Execute({}, &info)); + ARROW_RETURN_NOT_OK(PrintResults(sqlClient, call_options, info)); + ARROW_RETURN_NOT_OK(prepared_statement->Close({})); } else if (fLS::FLAGS_command == "GetSchemas") { ARROW_RETURN_NOT_OK(sqlClient.GetSchemas(call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, &info)); @@ -153,6 +160,7 @@ Status RunMain() { } if (info != NULLPTR) { + // TODO Since we are closing the preparedStament above, this wont work. return PrintResults(sqlClient, call_options, info); } From 9414a2ef4a054ced35ed7c49dd9963db879dfa01 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 14:59:05 -0300 Subject: [PATCH 092/237] Fix checkstyle --- cpp/src/arrow/flight/flight-sql/client.h | 15 ++++---- .../arrow/flight/flight-sql/client_test.cc | 34 +++++++++---------- 2 files changed, 23 insertions(+), 26 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 0b6b6cc48d8..49d4e964702 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -18,13 +18,12 @@ #ifndef ARROW_FLIGHT_SQL_CLIENT_H #define ARROW_FLIGHT_SQL_CLIENT_H +#include #include +#include #include #include #include -#include -#include - namespace pb = arrow::flight::protocol; @@ -39,18 +38,19 @@ class ARROW_EXPORT PreparedStatementT { bool is_closed; T* client; -public: + public: /// \brief Constructor for the PreparedStatement class. /// \param[in] client A raw pointer to FlightClient. /// \param[in] query The query that will be executed. - PreparedStatementT(T* client, const std::string& query, std::shared_ptr result_buffer); + PreparedStatementT(T* client, const std::string& query, + std::shared_ptr result_buffer); /// \brief Executes the prepared statement query on the server. /// \param[in] call_options RPC-layer hints for this call. /// \param[out] info A FlightInfo object representing the stream(s) to fetch. /// \return Status. Status Execute(const FlightCallOptions& call_options, - std::unique_ptr *info); + std::unique_ptr* info); /// \brief Closes the prepared statement. /// \param[in] options RPC-layer hints for this call. @@ -179,7 +179,7 @@ class FlightSqlClientT { Status Prepare(const FlightCallOptions& options, const std::string& query, std::shared_ptr>* prepared_statement); -private: + private: std::unique_ptr client; }; @@ -188,7 +188,6 @@ class FlightSqlClientT { using FlightSqlClient = internal::FlightSqlClientT<>; using PreparedStatement = internal::PreparedStatementT<>; - } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 52fdfe80770..b0909d5745f 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -45,7 +45,7 @@ class FlightClientMock { std::unique_ptr*)); MOCK_METHOD(Status, DoAction, (const FlightCallOptions& options, const Action& action, - std::unique_ptr* results)); + std::unique_ptr* results)); }; class FlightMetadataReaderMock : public FlightMetadataReader { @@ -236,7 +236,7 @@ TEST(TestFlightSqlClient, TestExecute) { } TEST(TestFlightSqlClient, TestPreparedStatementExecute) { - auto *client_mock = new FlightClientMock(); + auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sqlClient(client_mock_ptr); FlightCallOptions call_options; @@ -244,33 +244,31 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecute) { const std::string query = "query"; ON_CALL(*client_mock, DoAction) - .WillByDefault([](const FlightCallOptions& options, const Action& action, - std::unique_ptr* results) { - google::protobuf::Any command; + .WillByDefault([](const FlightCallOptions& options, const Action& action, + std::unique_ptr* results) { + google::protobuf::Any command; - pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; - prepared_statement_result.set_prepared_statement_handle("query"); + prepared_statement_result.set_prepared_statement_handle("query"); - command.PackFrom(prepared_statement_result); + command.PackFrom(prepared_statement_result); - *results = std::unique_ptr( - new SimpleResultStream({Result{Buffer::FromString(command.SerializeAsString())}})); + *results = std::unique_ptr(new SimpleResultStream( + {Result{Buffer::FromString(command.SerializeAsString())}})); - return Status::OK(); - }); + return Status::OK(); + }); std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, - DoAction(_, _, _)); + EXPECT_CALL(*client_mock, DoAction(_, _, _)); std::shared_ptr> preparedStatement; - sqlClient.Prepare({}, query, &preparedStatement); + (void)sqlClient.Prepare({}, query, &preparedStatement); - EXPECT_CALL(*client_mock, - GetFlightInfo(_, _, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(_, _, &flight_info)); - (void) preparedStatement->Execute({}, &flight_info); + (void)preparedStatement->Execute({}, &flight_info); } TEST(TestFlightSqlClient, TestExecuteUpdate) { From a46dd6db945744c0b7f7ae81cf4abc6a07475f2d Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 14:59:15 -0300 Subject: [PATCH 093/237] Adding missing files from rebase --- cpp/src/arrow/flight/protocol_internal.cc | 26 +++++++++++++++++++++ cpp/src/arrow/flight/protocol_internal.h | 28 +++++++++++++++++++++++ 2 files changed, 54 insertions(+) create mode 100644 cpp/src/arrow/flight/protocol_internal.cc create mode 100644 cpp/src/arrow/flight/protocol_internal.h diff --git a/cpp/src/arrow/flight/protocol_internal.cc b/cpp/src/arrow/flight/protocol_internal.cc new file mode 100644 index 00000000000..9f815398e48 --- /dev/null +++ b/cpp/src/arrow/flight/protocol_internal.cc @@ -0,0 +1,26 @@ +// 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 + +#include "arrow/flight/protocol_internal.h" + +// NOTE(wesm): Including .cc files in another .cc file would ordinarily be a +// no-no. We have customized the serialization path for FlightData, which is +// currently only possible through some pre-processor commands that need to be +// included before either of these files is compiled. Because we don't want to +// edit the generated C++ files, we include them here and do our gRPC +// customizations in protocol-internal.h +#include "arrow/flight/Flight.grpc.pb.cc" // NOLINT +#include "arrow/flight/Flight.pb.cc" // NOLINT diff --git a/cpp/src/arrow/flight/protocol_internal.h b/cpp/src/arrow/flight/protocol_internal.h new file mode 100644 index 00000000000..98bf9238809 --- /dev/null +++ b/cpp/src/arrow/flight/protocol_internal.h @@ -0,0 +1,28 @@ +// 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 + +#pragma once + +// This addresses platform-specific defines, e.g. on Windows +#include "arrow/flight/platform.h" // IWYU pragma: keep + +// This header holds the Flight protobuf definitions. + +// Need to include this first to get our gRPC customizations +#include "arrow/flight/customize_protobuf.h" // IWYU pragma: export + +#include "arrow/flight/Flight.grpc.pb.h" // IWYU pragma: export +#include "arrow/flight/Flight.pb.h" // IWYU pragma: export From 297b7f06968e859149b6adca4aa4d4f7d5e2f637 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 15:09:04 -0300 Subject: [PATCH 094/237] Treat if the statement is already closed --- cpp/src/arrow/flight/flight-sql/client_impl.h | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 5524571d36e..86b15c6d9ff 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -268,9 +268,8 @@ PreparedStatementT::PreparedStatementT(T* client_, const std::string& query, template Status PreparedStatementT::Execute(const FlightCallOptions& call_options, std::unique_ptr* info) { - //TODO Treat this error better if (is_closed) { - std::cout << "Statement is closed" << std::endl; + return Status::Invalid("Statement already closed."); } pb::sql::CommandPreparedStatementQuery prepared_statement_query; @@ -297,17 +296,8 @@ bool PreparedStatementT::IsClosed() const { template Status PreparedStatementT::Close(const FlightCallOptions& options) { if (is_closed) { - return Status::OK(); + return Status::Invalid("Statement already closed."); } - - ActionType actionType; - actionType.type = "ClosePreparedStatement"; - actionType.description = - "Closes a reusable prepared statement resource on the server. " - "Request Message: ActionClosePreparedStatementRequest Response Message: N/A;"; - - Action action; - action.type = actionType.type; google::protobuf::Any command; pb::sql::ActionClosePreparedStatementRequest request; request.set_prepared_statement_handle( @@ -315,6 +305,8 @@ Status PreparedStatementT::Close(const FlightCallOptions& options) { command.PackFrom(request); + Action action; + action.type = "ClosePreparedStatement"; action.body = Buffer::FromString(command.SerializeAsString()); std::unique_ptr results; From 4b0adb6cabf9f357ca7afe53ea64d5b8a0aa262d Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 15:13:30 -0300 Subject: [PATCH 095/237] Fix test_app for prepared statement execution --- cpp/src/arrow/flight/flight-sql/test_app.cc | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index ebcf32d792a..5cd46e4af0f 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -129,7 +129,7 @@ Status RunMain() { ARROW_RETURN_NOT_OK(sqlClient.Execute(call_options, fLS::FLAGS_query, &info)); } else if (fLS::FLAGS_command == "GetCatalogs") { ARROW_RETURN_NOT_OK(sqlClient.GetCatalogs(call_options, &info)); - } else if (fLS::FLAGS_command == "Prepared") { + } else if (fLS::FLAGS_command == "PreparedStatementExecute") { std::shared_ptr prepared_statement; ARROW_RETURN_NOT_OK(sqlClient.Prepare({}, fLS::FLAGS_query, &prepared_statement)); @@ -159,8 +159,7 @@ Status RunMain() { call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, fLS::FLAGS_table, &info)); } - if (info != NULLPTR) { - // TODO Since we are closing the preparedStament above, this wont work. + if (info != NULLPTR && fLS::FLAGS_command != "PreparedStatementExecute") { return PrintResults(sqlClient, call_options, info); } From f80bfafaa4b93513010b0fe4651720a87f6f8a25 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 16:28:10 -0300 Subject: [PATCH 096/237] Pass options to the client calls --- cpp/src/arrow/flight/flight-sql/client_impl.h | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 86b15c6d9ff..ca33e646af6 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -240,7 +240,7 @@ Status FlightSqlClientT::Prepare( std::unique_ptr results; - ARROW_RETURN_NOT_OK(client->DoAction({}, action, &results)); + ARROW_RETURN_NOT_OK(client->DoAction(options, action, &results)); std::unique_ptr result; ARROW_RETURN_NOT_OK(results->Next(&result)); @@ -283,9 +283,7 @@ Status PreparedStatementT::Execute(const FlightCallOptions& call_options, const std::string& string = any.SerializeAsString(); const FlightDescriptor& descriptor = FlightDescriptor::Command(string); - ARROW_RETURN_NOT_OK(client->GetFlightInfo({}, descriptor, info)); - - return Status::OK(); + return client->GetFlightInfo(call_options, descriptor, info); } template @@ -311,7 +309,7 @@ Status PreparedStatementT::Close(const FlightCallOptions& options) { std::unique_ptr results; - ARROW_RETURN_NOT_OK(client->DoAction({}, action, &results)); + ARROW_RETURN_NOT_OK(client->DoAction(options, action, &results)); is_closed = true; From 78b454094835bff1a829f3dec1e3cbb4a5d0fcf3 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 16:28:22 -0300 Subject: [PATCH 097/237] Modify the constructor of the PreparedStatement --- cpp/src/arrow/flight/flight-sql/client.h | 2 +- cpp/src/arrow/flight/flight-sql/client_impl.h | 22 ++++++++++--------- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 49d4e964702..0bfec4d0681 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -43,7 +43,7 @@ class ARROW_EXPORT PreparedStatementT { /// \param[in] client A raw pointer to FlightClient. /// \param[in] query The query that will be executed. PreparedStatementT(T* client, const std::string& query, - std::shared_ptr result_buffer); + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result); /// \brief Executes the prepared statement query on the server. /// \param[in] call_options RPC-layer hints for this call. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index ca33e646af6..94fb47097d5 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -245,23 +245,25 @@ Status FlightSqlClientT::Prepare( std::unique_ptr result; ARROW_RETURN_NOT_OK(results->Next(&result)); - prepared_statement->reset(new PreparedStatementT(client.get(), query, result->body)); - - return Status::OK(); -} - -template -PreparedStatementT::PreparedStatementT(T* client_, const std::string& query, - std::shared_ptr result_buffer) - : client(client_) { google::protobuf::Any prepared_result; - std::shared_ptr message = std::move(result_buffer); + std::shared_ptr message = std::move(result->body); prepared_result.ParseFromArray(message->data(), static_cast(message->size())); + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + prepared_result.UnpackTo(&prepared_statement_result); + prepared_statement->reset(new PreparedStatementT(client.get(), query, prepared_statement_result)); + + return Status::OK(); +} + +template +PreparedStatementT::PreparedStatementT(T* client_, const std::string& query, + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result_) + : client(client_), prepared_statement_result(std::move(prepared_statement_result_)) { is_closed = false; } From 8d60000190848035b3d138a3f92ea6ad331d2b3d Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 6 Oct 2021 16:42:19 -0300 Subject: [PATCH 098/237] Remove duplicate function due to rebase --- .../flight-sql/example/sqlite_server.cc | 27 ------------------- .../arrow/flight/flight-sql/sql_server.cpp | 11 -------- 2 files changed, 38 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index c5ea9622934..001e08c2faf 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -220,33 +220,6 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& com return Status::OK(); } -Status SQLiteFlightSqlServer::GetFlightInfoSchemas( - const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, - const FlightDescriptor& descriptor, std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, command, - SqlSchema::GetSchemasSchema()); -} - -Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& command, - const ServerCallContext& context, - std::unique_ptr* result) { - // As SQLite doesn't support schemas, this will return an empty record batch. - - const std::shared_ptr& schema = SqlSchema::GetSchemasSchema(); - - 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})); - *result = std::unique_ptr(new RecordBatchStream(reader)); - return Status::OK(); -} - Status SQLiteFlightSqlServer::GetFlightInfoTables( const pb::sql::CommandGetTables& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index 667f18b4c81..e9a03649a05 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -266,17 +266,6 @@ std::shared_ptr SqlSchema::GetTablesSchemaWithIncludedSchema() { field("table_schema", binary())}); } -std::shared_ptr SqlSchema::GetTablesSchema() { - return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8()), - field("table_name", utf8()), field("table_type", utf8())}); -} - -std::shared_ptr SqlSchema::GetTablesSchemaWithIncludedSchema() { - return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8()), - field("table_name", utf8()), field("table_type", utf8()), - field("table_schema", binary())}); -} - } // namespace sql } // namespace flight } // namespace arrow From de161ec9d10eb1a8cf563e4b33cce247f24fd764 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 13:49:17 -0300 Subject: [PATCH 099/237] Remove extra lines and duplicated import --- cpp/src/arrow/flight/flight-sql/client_impl.h | 1 - cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc | 2 -- 2 files changed, 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 94fb47097d5..70243023224 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -23,7 +23,6 @@ #include - namespace pb = arrow::flight::protocol; namespace arrow { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 001e08c2faf..38fe21e1b2c 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -18,8 +18,6 @@ #include #include -#include - #include "arrow/api.h" #include "arrow/flight/flight-sql/sql_server.h" #include "arrow/flight/flight-sql/example/sqlite_server.h" From d30cef29e2a51c2bc4d476471b0e7740dd029098 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 13:49:51 -0300 Subject: [PATCH 100/237] Move constructor to the top of the file --- cpp/src/arrow/flight/flight-sql/client_impl.h | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 70243023224..281a36b7441 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -35,6 +35,14 @@ FlightSqlClientT::FlightSqlClientT(std::unique_ptr& client) { this->client = std::move(client); } +template +PreparedStatementT::PreparedStatementT( + T* client_, const std::string& query, + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result_) + : client(client_), prepared_statement_result(std::move(prepared_statement_result_)) { + is_closed = false; +} + template FlightSqlClientT::~FlightSqlClientT() = default; @@ -259,13 +267,6 @@ Status FlightSqlClientT::Prepare( return Status::OK(); } -template -PreparedStatementT::PreparedStatementT(T* client_, const std::string& query, - pb::sql::ActionCreatePreparedStatementResult prepared_statement_result_) - : client(client_), prepared_statement_result(std::move(prepared_statement_result_)) { - is_closed = false; -} - template Status PreparedStatementT::Execute(const FlightCallOptions& call_options, std::unique_ptr* info) { From 8f23bd5953697a7559d9e8818f91ff65c9cd4969 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 13:50:24 -0300 Subject: [PATCH 101/237] rename the variable of the CommandPreparedStatementQuery --- cpp/src/arrow/flight/flight-sql/client_impl.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 281a36b7441..e9d52d99cff 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -274,13 +274,13 @@ Status PreparedStatementT::Execute(const FlightCallOptions& call_options, return Status::Invalid("Statement already closed."); } - pb::sql::CommandPreparedStatementQuery prepared_statement_query; + pb::sql::CommandPreparedStatementQuery execute_query_command; - prepared_statement_query.set_prepared_statement_handle( + execute_query_command.set_prepared_statement_handle( prepared_statement_result.prepared_statement_handle()); google::protobuf::Any any; - any.PackFrom(prepared_statement_query); + any.PackFrom(execute_query_command); const std::string& string = any.SerializeAsString(); const FlightDescriptor& descriptor = FlightDescriptor::Command(string); From 6759818b9567692dad9e6166a897b06041f98b66 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 14:04:10 -0300 Subject: [PATCH 102/237] Make the destructor call the Close method --- cpp/src/arrow/flight/flight-sql/client.h | 12 ++++++++++-- cpp/src/arrow/flight/flight-sql/client_impl.h | 14 +++++++++++--- cpp/src/arrow/flight/flight-sql/test_app.cc | 5 ++--- 3 files changed, 23 insertions(+), 8 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 0bfec4d0681..5618bb38b57 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -35,6 +35,7 @@ namespace internal { template class ARROW_EXPORT PreparedStatementT { pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + FlightCallOptions options; bool is_closed; T* client; @@ -42,8 +43,15 @@ class ARROW_EXPORT PreparedStatementT { /// \brief Constructor for the PreparedStatement class. /// \param[in] client A raw pointer to FlightClient. /// \param[in] query The query that will be executed. - PreparedStatementT(T* client, const std::string& query, - pb::sql::ActionCreatePreparedStatementResult prepared_statement_result); + PreparedStatementT( + T* client, const std::string& query, + pb::sql::ActionCreatePreparedStatementResult& prepared_statement_result, + const 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. + ~PreparedStatementT(); /// \brief Executes the prepared statement query on the server. /// \param[in] call_options RPC-layer hints for this call. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index e9d52d99cff..62aa217e8e6 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -38,14 +38,22 @@ FlightSqlClientT::FlightSqlClientT(std::unique_ptr& client) { template PreparedStatementT::PreparedStatementT( T* client_, const std::string& query, - pb::sql::ActionCreatePreparedStatementResult prepared_statement_result_) - : client(client_), prepared_statement_result(std::move(prepared_statement_result_)) { + pb::sql::ActionCreatePreparedStatementResult& prepared_statement_result_, + const FlightCallOptions& options_) + : client(client_), + prepared_statement_result(std::move(prepared_statement_result_)), + options(options_) { is_closed = false; } template FlightSqlClientT::~FlightSqlClientT() = default; +template +PreparedStatementT::~PreparedStatementT() { + Close(options); +} + inline FlightDescriptor GetFlightDescriptorForCommand( const google::protobuf::Message& command) { google::protobuf::Any any; @@ -262,7 +270,7 @@ Status FlightSqlClientT::Prepare( prepared_result.UnpackTo(&prepared_statement_result); - prepared_statement->reset(new PreparedStatementT(client.get(), query, prepared_statement_result)); + prepared_statement->reset(new PreparedStatementT(client.get(), query, prepared_statement_result, options)); return Status::OK(); } diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index 5cd46e4af0f..925a05a7d67 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -132,10 +132,9 @@ Status RunMain() { } else if (fLS::FLAGS_command == "PreparedStatementExecute") { std::shared_ptr prepared_statement; - ARROW_RETURN_NOT_OK(sqlClient.Prepare({}, fLS::FLAGS_query, &prepared_statement)); - ARROW_RETURN_NOT_OK(prepared_statement->Execute({}, &info)); + ARROW_RETURN_NOT_OK(sqlClient.Prepare(call_options, fLS::FLAGS_query, &prepared_statement)); + ARROW_RETURN_NOT_OK(prepared_statement->Execute(call_options, &info)); ARROW_RETURN_NOT_OK(PrintResults(sqlClient, call_options, info)); - ARROW_RETURN_NOT_OK(prepared_statement->Close({})); } else if (fLS::FLAGS_command == "GetSchemas") { ARROW_RETURN_NOT_OK(sqlClient.GetSchemas(call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, &info)); From a9361aa41958be9494da639a6bc495b31bf4120f Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 14:16:18 -0300 Subject: [PATCH 103/237] Remove options as parameter from PreparedStatement methods --- cpp/src/arrow/flight/flight-sql/client.h | 5 ++--- cpp/src/arrow/flight/flight-sql/client_impl.h | 9 ++++----- cpp/src/arrow/flight/flight-sql/test_app.cc | 2 +- 3 files changed, 7 insertions(+), 9 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 5618bb38b57..cb7ba88ae9a 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -57,13 +57,12 @@ class ARROW_EXPORT PreparedStatementT { /// \param[in] call_options RPC-layer hints for this call. /// \param[out] info A FlightInfo object representing the stream(s) to fetch. /// \return Status. - Status Execute(const FlightCallOptions& call_options, - std::unique_ptr* info); + Status Execute(std::unique_ptr* info); /// \brief Closes the prepared statement. /// \param[in] options RPC-layer hints for this call. /// \return Status. - Status Close(const FlightCallOptions& options); + Status Close(); /// \brief Checks if the prepared statement is closed. /// \return The state of the prepared statement. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 62aa217e8e6..fb39ff5c17d 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -51,7 +51,7 @@ FlightSqlClientT::~FlightSqlClientT() = default; template PreparedStatementT::~PreparedStatementT() { - Close(options); + Close(); } inline FlightDescriptor GetFlightDescriptorForCommand( @@ -276,8 +276,7 @@ Status FlightSqlClientT::Prepare( } template -Status PreparedStatementT::Execute(const FlightCallOptions& call_options, - std::unique_ptr* info) { +Status PreparedStatementT::Execute(std::unique_ptr* info) { if (is_closed) { return Status::Invalid("Statement already closed."); } @@ -293,7 +292,7 @@ Status PreparedStatementT::Execute(const FlightCallOptions& call_options, const std::string& string = any.SerializeAsString(); const FlightDescriptor& descriptor = FlightDescriptor::Command(string); - return client->GetFlightInfo(call_options, descriptor, info); + return client->GetFlightInfo(options, descriptor, info); } template @@ -302,7 +301,7 @@ bool PreparedStatementT::IsClosed() const { } template -Status PreparedStatementT::Close(const FlightCallOptions& options) { +Status PreparedStatementT::Close() { if (is_closed) { return Status::Invalid("Statement already closed."); } diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index 925a05a7d67..428cc0af33e 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -133,7 +133,7 @@ Status RunMain() { std::shared_ptr prepared_statement; ARROW_RETURN_NOT_OK(sqlClient.Prepare(call_options, fLS::FLAGS_query, &prepared_statement)); - ARROW_RETURN_NOT_OK(prepared_statement->Execute(call_options, &info)); + ARROW_RETURN_NOT_OK(prepared_statement->Execute(&info)); ARROW_RETURN_NOT_OK(PrintResults(sqlClient, call_options, info)); } else if (fLS::FLAGS_command == "GetSchemas") { ARROW_RETURN_NOT_OK(sqlClient.GetSchemas(call_options, &fLS::FLAGS_catalog, From 3c3b928fd22ad1b35421eb7108dfa336f5bc3073 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 14:44:12 -0300 Subject: [PATCH 104/237] Make the mock object be called twice --- cpp/src/arrow/flight/flight-sql/client_test.cc | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index b0909d5745f..820a831eda3 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -261,14 +261,14 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecute) { }); std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, DoAction(_, _, _)); + EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); std::shared_ptr> preparedStatement; - (void)sqlClient.Prepare({}, query, &preparedStatement); + (void)sqlClient.Prepare(call_options, query, &preparedStatement); EXPECT_CALL(*client_mock, GetFlightInfo(_, _, &flight_info)); - (void)preparedStatement->Execute({}, &flight_info); + (void)preparedStatement->Execute(&flight_info); } TEST(TestFlightSqlClient, TestExecuteUpdate) { From f8f404ada1c5fd778a60912597c61f5f1a0d8bd4 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 7 Oct 2021 15:06:11 -0300 Subject: [PATCH 105/237] Remove duplicate import --- cpp/src/arrow/flight/flight-sql/sql_server_test.cc | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index e2fce861d7a..2bafa101d24 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -25,7 +25,6 @@ #include #include #include -#include #define unparen(...) __VA_ARGS__ #define DECLARE_ARRAY(ARRAY_NAME, TYPE_CLASS, DATA) \ From dcc8f7fb5f80d69abe78334dd0eb0ad770411772 Mon Sep 17 00:00:00 2001 From: Abner Eduardo Ferreira Date: Thu, 7 Oct 2021 15:37:01 -0300 Subject: [PATCH 106/237] Add GetSqlInfo on client side for FlightSQL --- cpp/src/arrow/flight/flight-sql/client.h | 18 ++++++++++++++ cpp/src/arrow/flight/flight-sql/client_impl.h | 16 +++++++++++++ .../arrow/flight/flight-sql/client_test.cc | 24 +++++++++++++++++++ 3 files changed, 58 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index cb7ba88ae9a..b74bed35570 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -178,6 +178,24 @@ class FlightSqlClientT { Status GetTableTypes(const FlightCallOptions& options, std::unique_ptr* flight_info) const; + /// \brief Request a list of SQL information. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] sql_info the SQL info required. + /// \param[out] flight_info The FlightInfo describing where to access the dataset. + /// \return Status. + Status GetSqlInfo(const FlightCallOptions& options, + const std::vector& sql_info, + std::unique_ptr* flight_info) const; + + /// \brief Request a list of SQL information. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] sql_info the SQL info required. + /// \param[out] flight_info The FlightInfo describing where to access the dataset. + /// \return Status. + Status GetSqlInfo(const FlightCallOptions& options, + const std::vector& sql_info, + std::unique_ptr* flight_info) const; + /// \brief Create a prepared statement object. /// \param[in] options RPC-layer hints for this call. /// \param[in] query The query that will be executed. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index fb39ff5c17d..33f5933b673 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -325,6 +325,22 @@ Status PreparedStatementT::Close() { return Status::OK(); } +template +Status FlightSqlClientT::GetSqlInfo( + const FlightCallOptions& options, const std::vector& sql_info, + std::unique_ptr* flight_info) const { + pb::sql::CommandGetSqlInfo command; + for (const int& info : sql_info) command.add_info(info); + return GetFlightInfoForCommand(client, options, flight_info, command); +} + +template +Status FlightSqlClientT::GetSqlInfo( + const FlightCallOptions& options, const std::vector& sql_info, + std::unique_ptr* flight_info) const { + return GetSqlInfo(options, reinterpret_cast&>(sql_info), flight_info); +} + } // namespace internal } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 820a831eda3..64809a49e39 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -317,6 +317,30 @@ TEST(TestFlightSqlClient, TestExecuteUpdate) { ASSERT_EQ(num_rows, 100); } + +TEST(TestFlightSqlClient, TestGetSqlInfo) { + auto* client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sql_client(client_mock_ptr); + + 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}; + std::unique_ptr flight_info; + 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()); + + FlightCallOptions call_options; + EXPECT_CALL(*client_mock, + GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + (void) sql_client.GetSqlInfo(call_options, sql_info, &flight_info); +} + } // namespace sql } // namespace flight } // namespace arrow From ac77dbc9b310271b8ec578d96299e8b05dc4cb0b Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 7 Oct 2021 17:52:07 -0300 Subject: [PATCH 107/237] Change SqliteStatement.Reset argumento to pointer --- cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc | 6 +++--- cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h | 2 +- .../flight-sql/example/sqlite_statement_batch_reader.cc | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index 459389c0f2b..1774cf69f96 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -83,9 +83,9 @@ Status SqliteStatement::Step(int* rc) { return Status::OK(); } -Status SqliteStatement::Reset(int& rc) { - rc = sqlite3_reset(stmt_); - if (rc == SQLITE_ERROR) { +Status SqliteStatement::Reset(int* rc) { + *rc = sqlite3_reset(stmt_); + if (*rc == SQLITE_ERROR) { return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db_)); } diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h index 311f279f0ad..5c573f2df9d 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h @@ -51,7 +51,7 @@ class SqliteStatement { /// \brief Reset the state of the sqlite3_stmt. /// \param[out] rc The resulting return code from SQLite. /// \return Status. - Status Reset(int& rc); + Status Reset(int* rc); /// \brief Returns the underlying sqlite3_stmt. sqlite3_stmt* GetSqlite3Stmt(); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index 6003c1b9870..b29be607efe 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -115,7 +115,7 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr* out) { } if (!already_executed) { - ARROW_RETURN_NOT_OK(statement_->Reset(rc)); + ARROW_RETURN_NOT_OK(statement_->Reset(&rc)); ARROW_RETURN_NOT_OK(statement_->Step(&rc)); already_executed = true; } From 08b029434dcae677e27f53fe4cc1d7fec02f0b7e Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 7 Oct 2021 14:20:27 -0300 Subject: [PATCH 108/237] Implement CommandStatementUpdate on server example --- cpp/src/arrow/flight/flight-sql/client_impl.h | 3 ++- .../flight-sql/example/sqlite_server.cc | 22 +++++++++++++++++ .../flight/flight-sql/example/sqlite_server.h | 4 ++++ .../flight-sql/example/sqlite_statement.cc | 9 +++++++ .../flight-sql/example/sqlite_statement.h | 5 ++++ .../arrow/flight/flight-sql/sql_server.cpp | 24 +++++++++++++++++++ cpp/src/arrow/flight/flight-sql/sql_server.h | 9 +++++++ .../flight/flight-sql/sql_server_test.cc | 21 ++++++++++++++++ 8 files changed, 96 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 33f5933b673..46e063816b8 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -91,9 +91,10 @@ Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); + std::unique_ptr writer; std::unique_ptr reader; - ARROW_RETURN_NOT_OK(client->DoPut(options, descriptor, NULLPTR, NULL, &reader)); + ARROW_RETURN_NOT_OK(client->DoPut(options, descriptor, NULLPTR, &writer, &reader)); std::shared_ptr metadata; diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 38fe21e1b2c..5e47f1a0024 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -265,6 +265,28 @@ Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& comma return Status::OK(); } +Status SQLiteFlightSqlServer::DoPutCommandStatementUpdate( + const pb::sql::CommandStatementUpdate& command, const ServerCallContext& context, + std::unique_ptr& reader, + std::unique_ptr& writer) { + const std::string& sql = command.query(); + + std::shared_ptr statement; + ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, sql, &statement)); + + pb::sql::DoPutUpdateResult result; + + int64_t record_count; + ARROW_RETURN_NOT_OK(statement->ExecuteUpdate(&record_count)); + + result.set_record_count(record_count); + + const std::shared_ptr& buffer = Buffer::FromString(result.SerializeAsString()); + ARROW_RETURN_NOT_OK(writer->WriteMetadata(*buffer)); + + return Status::OK(); +} + } // namespace example } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index 4464920f17b..d41c57a6c93 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -61,6 +61,10 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { Status DoGetSchemas(const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, std::unique_ptr* result) override; + Status DoPutCommandStatementUpdate( + const pb::sql::CommandStatementUpdate& update, const ServerCallContext& context, + std::unique_ptr& reader, + std::unique_ptr& writer) override; Status GetFlightInfoTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index 1774cf69f96..a024371d624 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -94,6 +94,15 @@ Status SqliteStatement::Reset(int* rc) { sqlite3_stmt* SqliteStatement::GetSqlite3Stmt() { return stmt_; } +Status SqliteStatement::ExecuteUpdate(int64_t* result) { + int rc; + ARROW_RETURN_NOT_OK(Step(&rc)); + + *result = sqlite3_changes(db_); + + return Status::OK(); +} + } // namespace example } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h index 5c573f2df9d..ee9191e42b3 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h @@ -56,6 +56,11 @@ class SqliteStatement { /// \brief Returns the underlying sqlite3_stmt. sqlite3_stmt* GetSqlite3Stmt(); + /// \brief Executes an UPDATE, INSERT or DELETE statement. + /// \param[out] result The number of rows changed by execution. + /// \return Status. + Status ExecuteUpdate(int64_t* result); + private: sqlite3* db_; sqlite3_stmt* stmt_; diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index e9a03649a05..dfb700df666 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -126,6 +126,23 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket 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; + any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size())); + + if (any.Is()) { + pb::sql::CommandStatementUpdate command; + any.UnpackTo(&command); + return DoPutCommandStatementUpdate(command, context, reader, writer); + } + + return Status::Invalid("The defined request is invalid."); +} + Status FlightSqlServerBase::GetFlightInfoCatalogs(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { @@ -246,6 +263,13 @@ Status FlightSqlServerBase::GetFlightInfoImportedKeys( return Status::NotImplemented("DoGetExportedKeys not implemented"); } +Status FlightSqlServerBase::DoPutCommandStatementUpdate( + const pb::sql::CommandStatementUpdate& command, const ServerCallContext& context, + std::unique_ptr& reader, + std::unique_ptr& writer) { + return Status::NotImplemented("DoPutCommandStatementUpdate not implemented"); +} + std::shared_ptr SqlSchema::GetCatalogsSchema() { return arrow::schema({field("catalog_name", utf8())}); } diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index e5b2a521597..4fa86fe68b4 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -39,6 +39,10 @@ class FlightSqlServerBase : public FlightServerBase { 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; + /// \brief Gets a FlightInfo for executing a SQL query. /// \param[in] command The CommandStatementQuery object containing the SQL /// statement. @@ -257,6 +261,11 @@ class FlightSqlServerBase : public FlightServerBase { virtual Status DoGetImportedKeys(const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, std::unique_ptr* result); + + virtual Status DoPutCommandStatementUpdate( + const pb::sql::CommandStatementUpdate& command, const ServerCallContext& context, + std::unique_ptr& reader, + std::unique_ptr& writer); }; /// \brief Auxiliary class containing all Schemas used on Flight SQL. diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 2bafa101d24..c8cc94ac7c1 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -285,6 +285,27 @@ TEST(TestFlightSqlServer, TestCommandGetSchemas) { ASSERT_EQ(0, table->num_rows()); } +TEST(TestFlightSqlServer, TestCommandStatementUpdate) { + int64_t result; + ASSERT_OK(sql_client->ExecuteUpdate( + {}, + "INSERT INTO intTable (keyName, value) VALUES " + "('KEYNAME1', 1001), ('KEYNAME2', 1002), ('KEYNAME3', 1003)", + &result)); + ASSERT_EQ(3, result); + + ASSERT_OK( + sql_client->ExecuteUpdate({}, + "UPDATE intTable SET keyName = 'KEYNAME1' " + "WHERE keyName = 'KEYNAME2' OR keyName = 'KEYNAME3'", + &result)); + ASSERT_EQ(2, result); + + ASSERT_OK(sql_client->ExecuteUpdate( + {}, "DELETE FROM intTable WHERE keyName = 'KEYNAME1'", &result)); + ASSERT_EQ(3, result); +} + auto env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); From ca738150306bab8866f4398749d697fdfddf7000 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 7 Oct 2021 17:49:05 -0300 Subject: [PATCH 109/237] Add missing docs for DoPutCommandStatementUpdate --- cpp/src/arrow/flight/flight-sql/sql_server.h | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 4fa86fe68b4..430fe7587ec 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -262,6 +262,12 @@ class FlightSqlServerBase : public FlightServerBase { const ServerCallContext& context, std::unique_ptr* result); + /// \brief Executes an update SQL statement. + /// \param[in] command The CommandStatementUpdate object containing the SQL statement. + /// \param[in] context The call context. + /// \param[in] reader a sequence of uploaded record batches. + /// \param[in] writer send metadata back to the client. + /// \return Status. virtual Status DoPutCommandStatementUpdate( const pb::sql::CommandStatementUpdate& command, const ServerCallContext& context, std::unique_ptr& reader, From 46c568171b8d0f45a19b6929e5fac7f6d605e64b Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 5 Oct 2021 13:14:55 -0300 Subject: [PATCH 110/237] Implement CommandGetSchemas --- .../flight-sql/example/sqlite_server.cc | 27 +++++++++++++++++++ cpp/src/arrow/flight/flight-sql/sql_server.h | 1 + 2 files changed, 28 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 5e47f1a0024..d641d53a3e3 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -218,6 +218,33 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& com return Status::OK(); } +Status SQLiteFlightSqlServer::GetFlightInfoSchemas( + const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return GetFlightInfoForCommand(descriptor, info, command, + SqlSchema::GetSchemasSchema()); +} + +Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& command, + const ServerCallContext& context, + std::unique_ptr* result) { + // As SQLite doesn't support schemas, this will return an empty record batch. + + const std::shared_ptr& schema = SqlSchema::GetSchemasSchema(); + + 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})); + *result = std::unique_ptr(new RecordBatchStream(reader)); + return Status::OK(); +} + Status SQLiteFlightSqlServer::GetFlightInfoTables( const pb::sql::CommandGetTables& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 430fe7587ec..3f927e50dd8 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -277,6 +277,7 @@ class FlightSqlServerBase : public FlightServerBase { /// \brief Auxiliary class containing all Schemas used on Flight SQL. class SqlSchema { public: + /// \brief Gets the Schema used on CommandGetCatalogs response. /// \return The default schema template. static std::shared_ptr GetCatalogsSchema(); From e85e8f84e04a1e6fb60672ae352481fd45a8a3fc Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 30 Sep 2021 15:39:18 -0300 Subject: [PATCH 111/237] Fix checkstyle --- cpp/src/arrow/flight/flight-sql/sql_server.h | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 3f927e50dd8..430fe7587ec 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -277,7 +277,6 @@ class FlightSqlServerBase : public FlightServerBase { /// \brief Auxiliary class containing all Schemas used on Flight SQL. class SqlSchema { public: - /// \brief Gets the Schema used on CommandGetCatalogs response. /// \return The default schema template. static std::shared_ptr GetCatalogsSchema(); From e98fd67350e7c8839a1bdc361cb55c07a026e501 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 13:29:25 -0300 Subject: [PATCH 112/237] Add methods to set and get parameters --- cpp/src/arrow/flight/flight-sql/client.h | 5 +++ cpp/src/arrow/flight/flight-sql/client_impl.h | 39 ++++++++++++++++++- 2 files changed, 43 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index b74bed35570..adfbd9c937c 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -35,6 +35,7 @@ namespace internal { template class ARROW_EXPORT PreparedStatementT { pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + std::shared_ptr parameter_binding; FlightCallOptions options; bool is_closed; T* client; @@ -59,6 +60,10 @@ class ARROW_EXPORT PreparedStatementT { /// \return Status. Status Execute(std::unique_ptr* info); + //TODO Add doc to these methods + Status GetParameterSchema(std::shared_ptr* schema); + Status SetParameters(const std::shared_ptr& parameter_binding_); + /// \brief Closes the prepared statement. /// \param[in] options RPC-layer hints for this call. /// \return Status. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 46e063816b8..8081ab5918b 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -20,8 +20,12 @@ #include #include #include +#include "arrow/ipc/reader.h" +#include + #include +#include namespace pb = arrow::flight::protocol; @@ -291,16 +295,49 @@ Status PreparedStatementT::Execute(std::unique_ptr* info) { any.PackFrom(execute_query_command); const std::string& string = any.SerializeAsString(); - const FlightDescriptor& descriptor = FlightDescriptor::Command(string); + const FlightDescriptor descriptor = FlightDescriptor::Command(string); + + if (parameter_binding && parameter_binding->num_rows() > 0) { + std::unique_ptr writer; + std::unique_ptr reader; + client->DoPut(descriptor, parameter_binding->schema(), &writer, &reader); + + std::shared_ptr buffer; + writer->WriteRecordBatch(*parameter_binding); + writer->DoneWriting(); + reader->ReadMetadata(&buffer); + } return client->GetFlightInfo(options, descriptor, info); } +template +Status PreparedStatementT::SetParameters(const std::shared_ptr& parameter_binding_) { + parameter_binding = parameter_binding_; + + return Status::OK(); +} + template bool PreparedStatementT::IsClosed() const { return is_closed; } +template +Status PreparedStatementT::GetParameterSchema(std::shared_ptr* schema) { + //TODO improve this function + ipc::DictionaryMemo in_memo; + + auto &args = prepared_statement_result.parameter_schema(); + std::shared_ptr schema_buffer = std::make_shared(args); + + io::BufferReader reader(schema_buffer); + + ARROW_ASSIGN_OR_RAISE(*schema, ReadSchema(&reader, &in_memo)) + + return Status::OK(); +} + template Status PreparedStatementT::Close() { if (is_closed) { From 832aec33f9b3667e7d60681a1acd70f3bcdb26d6 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 13:29:57 -0300 Subject: [PATCH 113/237] add paremter binding to query execution with prepared statement --- cpp/src/arrow/flight/flight-sql/test_app.cc | 35 ++++++++++++++++++++- 1 file changed, 34 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index 428cc0af33e..a523170f44c 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -25,6 +25,7 @@ #include #include +#include using arrow::Result; using arrow::Schema; @@ -135,7 +136,39 @@ Status RunMain() { ARROW_RETURN_NOT_OK(sqlClient.Prepare(call_options, fLS::FLAGS_query, &prepared_statement)); ARROW_RETURN_NOT_OK(prepared_statement->Execute(&info)); ARROW_RETURN_NOT_OK(PrintResults(sqlClient, call_options, info)); - } else if (fLS::FLAGS_command == "GetSchemas") { + } else if (fLS::FLAGS_command == "PreparedStatementExecuteUpdate") { + std::shared_ptr prepared_statement; + ARROW_RETURN_NOT_OK(sqlClient.Prepare({}, fLS::FLAGS_query, &prepared_statement)); + std::shared_ptr schema; + ARROW_RETURN_NOT_OK(prepared_statement->GetParameterSchema(&schema)); + +// arrow::StringBuilder string_builder; + arrow::Int64Builder int_builder; + +// ARROW_RETURN_NOT_OK(string_builder.Append("jose")); + ARROW_RETURN_NOT_OK(int_builder.Append(1)); + +// std::shared_ptr string_array; + std::shared_ptr int_array; + +// ARROW_RETURN_NOT_OK(string_builder.Finish(&string_array)); + ARROW_RETURN_NOT_OK(int_builder.Finish(&int_array)); + + std::shared_ptr result; + +// std::vector> arrays(builders.size()); +// for (int i = 0; i < num_fields; i++) { +// ARROW_RETURN_NOT_OK(builders[i]->Finish(&arrays[i])); +// } +// + result = arrow::RecordBatch::Make(schema, 1, {int_array}); + + ARROW_RETURN_NOT_OK(prepared_statement->SetParameters(result)); + ARROW_RETURN_NOT_OK(prepared_statement->Execute(call_options, &info)); + ARROW_RETURN_NOT_OK(PrintResults(sqlClient, call_options, info)); + + std::cout << 1 << std::endl; + }else if (fLS::FLAGS_command == "GetSchemas") { ARROW_RETURN_NOT_OK(sqlClient.GetSchemas(call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, &info)); } else if (fLS::FLAGS_command == "GetTableTypes") { From 7f9daa9f467461c3b4010417588131c05b8d8fa8 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 14:49:00 -0300 Subject: [PATCH 114/237] remove unused code from test_app --- cpp/src/arrow/flight/flight-sql/test_app.cc | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index a523170f44c..8b4618058a6 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -142,32 +142,16 @@ Status RunMain() { std::shared_ptr schema; ARROW_RETURN_NOT_OK(prepared_statement->GetParameterSchema(&schema)); -// arrow::StringBuilder string_builder; arrow::Int64Builder int_builder; - -// ARROW_RETURN_NOT_OK(string_builder.Append("jose")); ARROW_RETURN_NOT_OK(int_builder.Append(1)); - -// std::shared_ptr string_array; std::shared_ptr int_array; - -// ARROW_RETURN_NOT_OK(string_builder.Finish(&string_array)); ARROW_RETURN_NOT_OK(int_builder.Finish(&int_array)); - std::shared_ptr result; - -// std::vector> arrays(builders.size()); -// for (int i = 0; i < num_fields; i++) { -// ARROW_RETURN_NOT_OK(builders[i]->Finish(&arrays[i])); -// } -// result = arrow::RecordBatch::Make(schema, 1, {int_array}); ARROW_RETURN_NOT_OK(prepared_statement->SetParameters(result)); ARROW_RETURN_NOT_OK(prepared_statement->Execute(call_options, &info)); ARROW_RETURN_NOT_OK(PrintResults(sqlClient, call_options, info)); - - std::cout << 1 << std::endl; }else if (fLS::FLAGS_command == "GetSchemas") { ARROW_RETURN_NOT_OK(sqlClient.GetSchemas(call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, &info)); From 9ccb72e90d2bee8d77b704156f99f2785665191e Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 14:49:28 -0300 Subject: [PATCH 115/237] remove TODO --- cpp/src/arrow/flight/flight-sql/client_impl.h | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 8081ab5918b..42525e060b5 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -325,7 +325,6 @@ bool PreparedStatementT::IsClosed() const { template Status PreparedStatementT::GetParameterSchema(std::shared_ptr* schema) { - //TODO improve this function ipc::DictionaryMemo in_memo; auto &args = prepared_statement_result.parameter_schema(); From b2b1b8c3c854048ef793160e69a8f098ab4133a0 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 7 Oct 2021 14:50:34 -0300 Subject: [PATCH 116/237] Add macro ARRROW_RETURN_NOT_OK --- cpp/src/arrow/flight/flight-sql/client_impl.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 42525e060b5..f5ed6294513 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -303,9 +303,9 @@ Status PreparedStatementT::Execute(std::unique_ptr* info) { client->DoPut(descriptor, parameter_binding->schema(), &writer, &reader); std::shared_ptr buffer; - writer->WriteRecordBatch(*parameter_binding); - writer->DoneWriting(); - reader->ReadMetadata(&buffer); + ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*parameter_binding)); + ARROW_RETURN_NOT_OK(writer->DoneWriting()); + ARROW_RETURN_NOT_OK(reader->ReadMetadata(&buffer)); } return client->GetFlightInfo(options, descriptor, info); From b19552ee18036912f21994f20349fb831206b345 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 8 Oct 2021 13:25:35 -0300 Subject: [PATCH 117/237] Fix add a break line --- cpp/src/arrow/flight/flight-sql/client_impl.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index f5ed6294513..a3a1df9f6e6 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -275,7 +275,8 @@ Status FlightSqlClientT::Prepare( prepared_result.UnpackTo(&prepared_statement_result); - prepared_statement->reset(new PreparedStatementT(client.get(), query, prepared_statement_result, options)); + prepared_statement->reset( + new PreparedStatementT(client.get(), query, prepared_statement_result, options)); return Status::OK(); } From 15a6b3d17ce168a30c83049185bd9a97c7ad5311 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 8 Oct 2021 13:26:00 -0300 Subject: [PATCH 118/237] Pass option to DoPut CALL --- cpp/src/arrow/flight/flight-sql/client_impl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index a3a1df9f6e6..e6e49d651c6 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -301,7 +301,7 @@ Status PreparedStatementT::Execute(std::unique_ptr* info) { if (parameter_binding && parameter_binding->num_rows() > 0) { std::unique_ptr writer; std::unique_ptr reader; - client->DoPut(descriptor, parameter_binding->schema(), &writer, &reader); + client->DoPut(options, descriptor, parameter_binding->schema(), &writer, &reader); std::shared_ptr buffer; ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*parameter_binding)); From b59afc612594fdcfa64f1d633c263856f044d25b Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 8 Oct 2021 13:27:11 -0300 Subject: [PATCH 119/237] Create a mocked client test for parameter binding --- .../arrow/flight/flight-sql/client_test.cc | 114 ++++++++++++++++++ 1 file changed, 114 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 64809a49e39..02e7dd2e1e1 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -23,6 +23,8 @@ #include #include +#include + namespace pb = arrow::flight::protocol; using ::testing::_; using ::testing::Ref; @@ -62,6 +64,45 @@ class FlightMetadataReaderMock : public FlightMetadataReader { } }; +class FlightStreamWriterMock : public FlightStreamWriter { +public: + explicit 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); @@ -271,6 +312,79 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecute) { (void)preparedStatement->Execute(&flight_info); } +TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { + auto* client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sqlClient(client_mock_ptr); + FlightCallOptions call_options; + + const std::string query = "query"; + + ON_CALL(*client_mock, 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(*client_mock, 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(); + }); + + + std::unique_ptr flight_info; + EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); + EXPECT_CALL(*client_mock, DoPut(_, _, _, _, _)); + + std::shared_ptr> prepared_statement; + (void)sqlClient.Prepare(call_options, query, &prepared_statement); + + std::shared_ptr schema; + (void)prepared_statement->GetParameterSchema(&schema); + + arrow::Int64Builder int_builder; + (void)int_builder.Append(1); + std::shared_ptr int_array; + (void)int_builder.Finish(&int_array); + std::shared_ptr result; + result = arrow::RecordBatch::Make(schema, 1, {int_array}); + (void)prepared_statement->SetParameters(result); + + EXPECT_CALL(*client_mock, GetFlightInfo(_, _, &flight_info)); + + (void)prepared_statement->Execute(&flight_info); +} + TEST(TestFlightSqlClient, TestExecuteUpdate) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); From e3facf744f7ad0d4466a28121a522a8542020387 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 8 Oct 2021 13:27:35 -0300 Subject: [PATCH 120/237] Add prepared statement parameter binding to test app --- cpp/src/arrow/flight/flight-sql/test_app.cc | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index 8b4618058a6..b0601d84de0 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -26,6 +26,7 @@ #include #include #include +#include using arrow::Result; using arrow::Schema; @@ -136,7 +137,7 @@ Status RunMain() { ARROW_RETURN_NOT_OK(sqlClient.Prepare(call_options, fLS::FLAGS_query, &prepared_statement)); ARROW_RETURN_NOT_OK(prepared_statement->Execute(&info)); ARROW_RETURN_NOT_OK(PrintResults(sqlClient, call_options, info)); - } else if (fLS::FLAGS_command == "PreparedStatementExecuteUpdate") { + } else if (fLS::FLAGS_command == "PreparedStatementExecuteParameterBinding") { std::shared_ptr prepared_statement; ARROW_RETURN_NOT_OK(sqlClient.Prepare({}, fLS::FLAGS_query, &prepared_statement)); std::shared_ptr schema; @@ -150,7 +151,7 @@ Status RunMain() { result = arrow::RecordBatch::Make(schema, 1, {int_array}); ARROW_RETURN_NOT_OK(prepared_statement->SetParameters(result)); - ARROW_RETURN_NOT_OK(prepared_statement->Execute(call_options, &info)); + ARROW_RETURN_NOT_OK(prepared_statement->Execute(&info)); ARROW_RETURN_NOT_OK(PrintResults(sqlClient, call_options, info)); }else if (fLS::FLAGS_command == "GetSchemas") { ARROW_RETURN_NOT_OK(sqlClient.GetSchemas(call_options, &fLS::FLAGS_catalog, @@ -175,7 +176,8 @@ Status RunMain() { call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, fLS::FLAGS_table, &info)); } - if (info != NULLPTR && fLS::FLAGS_command != "PreparedStatementExecute") { + if (info != NULLPTR && + !boost::istarts_with(fLS::FLAGS_command, "PreparedStatementExecute")) { return PrintResults(sqlClient, call_options, info); } From eb2f76ce63cbbe514288886f9c8ab5c0157971c9 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 8 Oct 2021 13:39:56 -0300 Subject: [PATCH 121/237] Add missing docs to prepared statement methods --- cpp/src/arrow/flight/flight-sql/client.h | 8 +++++- .../flight-sql/example/sqlite_server.cc | 27 ------------------- 2 files changed, 7 insertions(+), 28 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index adfbd9c937c..77c7fdd0ac5 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -60,8 +60,14 @@ class ARROW_EXPORT PreparedStatementT { /// \return Status. Status Execute(std::unique_ptr* info); - //TODO Add doc to these methods + /// \brief Retrieve the parameter schema from the query. + /// \param schema The parameter schema from the query. + /// \return Status. Status GetParameterSchema(std::shared_ptr* schema); + + /// \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(const std::shared_ptr& parameter_binding_); /// \brief Closes the prepared statement. diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index d641d53a3e3..5e47f1a0024 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -218,33 +218,6 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& com return Status::OK(); } -Status SQLiteFlightSqlServer::GetFlightInfoSchemas( - const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, - const FlightDescriptor& descriptor, std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, command, - SqlSchema::GetSchemasSchema()); -} - -Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& command, - const ServerCallContext& context, - std::unique_ptr* result) { - // As SQLite doesn't support schemas, this will return an empty record batch. - - const std::shared_ptr& schema = SqlSchema::GetSchemasSchema(); - - 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})); - *result = std::unique_ptr(new RecordBatchStream(reader)); - return Status::OK(); -} - Status SQLiteFlightSqlServer::GetFlightInfoTables( const pb::sql::CommandGetTables& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { From 432b0267573138134219e78d22fc9aea4e4c92f6 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 8 Oct 2021 14:21:26 -0300 Subject: [PATCH 122/237] Implement method GetResultSetSchema --- cpp/src/arrow/flight/flight-sql/client.h | 5 +++++ cpp/src/arrow/flight/flight-sql/client_impl.h | 14 ++++++++++++++ cpp/src/arrow/flight/flight-sql/test_app.cc | 9 ++++++--- 3 files changed, 25 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 77c7fdd0ac5..6116f0e8286 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -65,6 +65,11 @@ class ARROW_EXPORT PreparedStatementT { /// \return Status. Status GetParameterSchema(std::shared_ptr* schema); + /// \brief Retrieve the ResultSet schema from the query. + /// \param schema The ResultSet schema from the query. + /// \return Status. + Status GetResultSetSchema(std::shared_ptr* schema); + /// \brief Set a RecordBatch that contains the parameters that will be bind. /// \param parameter_binding_ The parameters that will be bind. /// \return Status. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index e6e49d651c6..f69e705fb5b 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -324,6 +324,20 @@ bool PreparedStatementT::IsClosed() const { return is_closed; } +template +Status PreparedStatementT::GetResultSetSchema(std::shared_ptr *schema) { + ipc::DictionaryMemo in_memo; + + auto &args = prepared_statement_result.dataset_schema(); + std::shared_ptr schema_buffer = std::make_shared(args); + + io::BufferReader reader(schema_buffer); + + ARROW_ASSIGN_OR_RAISE(*schema, ReadSchema(&reader, &in_memo)) + + return Status::OK(); +} + template Status PreparedStatementT::GetParameterSchema(std::shared_ptr* schema) { ipc::DictionaryMemo in_memo; diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index b0601d84de0..1d25c90bf3e 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -140,15 +140,18 @@ Status RunMain() { } else if (fLS::FLAGS_command == "PreparedStatementExecuteParameterBinding") { std::shared_ptr prepared_statement; ARROW_RETURN_NOT_OK(sqlClient.Prepare({}, fLS::FLAGS_query, &prepared_statement)); - std::shared_ptr schema; - ARROW_RETURN_NOT_OK(prepared_statement->GetParameterSchema(&schema)); + std::shared_ptr parameter_schema; + std::shared_ptr result_set_schema; + ARROW_RETURN_NOT_OK(prepared_statement->GetParameterSchema(¶meter_schema)); + ARROW_RETURN_NOT_OK(prepared_statement->GetResultSetSchema(&result_set_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(schema, 1, {int_array}); + result = arrow::RecordBatch::Make(parameter_schema, 1, {int_array}); ARROW_RETURN_NOT_OK(prepared_statement->SetParameters(result)); ARROW_RETURN_NOT_OK(prepared_statement->Execute(&info)); From 8327b47cb50c3808e0ed06fc63ac99419efc3a35 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 8 Oct 2021 14:21:43 -0300 Subject: [PATCH 123/237] change (void) to ASSERT_OK on test --- cpp/src/arrow/flight/flight-sql/client_test.cc | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 02e7dd2e1e1..eb00eaf5b62 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -367,22 +367,22 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { EXPECT_CALL(*client_mock, DoPut(_, _, _, _, _)); std::shared_ptr> prepared_statement; - (void)sqlClient.Prepare(call_options, query, &prepared_statement); + ASSERT_OK(sqlClient.Prepare(call_options, query, &prepared_statement)); - std::shared_ptr schema; - (void)prepared_statement->GetParameterSchema(&schema); + std::shared_ptr parameter_schema; + ASSERT_OK(prepared_statement->GetParameterSchema(¶meter_schema)); arrow::Int64Builder int_builder; - (void)int_builder.Append(1); + ASSERT_OK(int_builder.Append(1)); std::shared_ptr int_array; - (void)int_builder.Finish(&int_array); + ASSERT_OK(int_builder.Finish(&int_array)); std::shared_ptr result; - result = arrow::RecordBatch::Make(schema, 1, {int_array}); - (void)prepared_statement->SetParameters(result); + result = arrow::RecordBatch::Make(parameter_schema, 1, {int_array}); + ASSERT_OK(prepared_statement->SetParameters(result)); EXPECT_CALL(*client_mock, GetFlightInfo(_, _, &flight_info)); - (void)prepared_statement->Execute(&flight_info); + ASSERT_OK(prepared_statement->Execute(&flight_info)); } TEST(TestFlightSqlClient, TestExecuteUpdate) { From 2f1c93585632e62e6baf73c6f0d5f977015eca31 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 8 Oct 2021 15:15:54 -0300 Subject: [PATCH 124/237] change const references from setParameters method --- cpp/src/arrow/flight/flight-sql/client.h | 2 +- cpp/src/arrow/flight/flight-sql/client_impl.h | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 6116f0e8286..9fbc8c8d1bf 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -73,7 +73,7 @@ class ARROW_EXPORT PreparedStatementT { /// \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(const std::shared_ptr& parameter_binding_); + Status SetParameters(std::shared_ptr parameter_binding); /// \brief Closes the prepared statement. /// \param[in] options RPC-layer hints for this call. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index f69e705fb5b..29df474a671 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -313,8 +313,8 @@ Status PreparedStatementT::Execute(std::unique_ptr* info) { } template -Status PreparedStatementT::SetParameters(const std::shared_ptr& parameter_binding_) { - parameter_binding = parameter_binding_; +Status PreparedStatementT::SetParameters(std::shared_ptr parameter_binding_) { + parameter_binding = std::move(parameter_binding_); return Status::OK(); } From 51462791dcb003b4b2e56795403577e646ef78b7 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 8 Oct 2021 15:16:28 -0300 Subject: [PATCH 125/237] Sort includes alphabetically --- cpp/src/arrow/flight/flight-sql/client_impl.h | 7 +++---- cpp/src/arrow/flight/flight-sql/test_app.cc | 4 ++-- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 29df474a671..605ac00f01b 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -18,14 +18,13 @@ #include #include #include +#include +#include +#include #include #include -#include "arrow/ipc/reader.h" -#include - #include -#include namespace pb = arrow::flight::protocol; diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index 1d25c90bf3e..a8bc19fa385 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +#include #include #include #include @@ -23,10 +24,9 @@ #include #include +#include #include #include -#include -#include using arrow::Result; using arrow::Schema; From afd0e3e8c6af317d23dbfb0ad64b9ee8c1aeaf21 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 8 Oct 2021 15:17:00 -0300 Subject: [PATCH 126/237] Add comment when calling ReadMetadata on DoPut --- cpp/src/arrow/flight/flight-sql/client_impl.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 605ac00f01b..4502a6cf0b2 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -302,9 +302,10 @@ Status PreparedStatementT::Execute(std::unique_ptr* info) { std::unique_ptr reader; client->DoPut(options, descriptor, parameter_binding->schema(), &writer, &reader); - std::shared_ptr buffer; 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)); } From 042fcc250c597bb03e95901b739251b68ff04557 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Fri, 8 Oct 2021 15:17:46 -0300 Subject: [PATCH 127/237] use variable close to its use --- cpp/src/arrow/flight/flight-sql/client_impl.h | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 4502a6cf0b2..2f3b104a606 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -326,13 +326,12 @@ bool PreparedStatementT::IsClosed() const { template Status PreparedStatementT::GetResultSetSchema(std::shared_ptr *schema) { - ipc::DictionaryMemo in_memo; - auto &args = prepared_statement_result.dataset_schema(); std::shared_ptr schema_buffer = std::make_shared(args); io::BufferReader reader(schema_buffer); + ipc::DictionaryMemo in_memo; ARROW_ASSIGN_OR_RAISE(*schema, ReadSchema(&reader, &in_memo)) return Status::OK(); @@ -340,13 +339,12 @@ Status PreparedStatementT::GetResultSetSchema(std::shared_ptr *schema template Status PreparedStatementT::GetParameterSchema(std::shared_ptr* schema) { - ipc::DictionaryMemo in_memo; - auto &args = prepared_statement_result.parameter_schema(); std::shared_ptr schema_buffer = std::make_shared(args); io::BufferReader reader(schema_buffer); + ipc::DictionaryMemo in_memo; ARROW_ASSIGN_OR_RAISE(*schema, ReadSchema(&reader, &in_memo)) return Status::OK(); From e8efe631e106c2dc9788af6db99899353a7b10d4 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 8 Oct 2021 15:57:04 -0300 Subject: [PATCH 128/237] Implement CommandGetTableTypes on server example --- .../flight-sql/example/sqlite_server.cc | 24 +++++++++++++++++++ .../flight/flight-sql/example/sqlite_server.h | 5 ++++ .../arrow/flight/flight-sql/sql_server.cpp | 4 ++++ cpp/src/arrow/flight/flight-sql/sql_server.h | 4 ++++ .../flight/flight-sql/sql_server_test.cc | 17 +++++++++++++ 5 files changed, 54 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 5e47f1a0024..12e767c9954 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -287,6 +287,30 @@ Status SQLiteFlightSqlServer::DoPutCommandStatementUpdate( return Status::OK(); } +Status SQLiteFlightSqlServer::GetFlightInfoTableTypes(const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { + pb::sql::CommandGetTableTypes command; + return GetFlightInfoForCommand(descriptor, info, command, + SqlSchema::GetTableTypesSchema()); +} + +Status SQLiteFlightSqlServer::DoGetTableTypes(const ServerCallContext& context, + std::unique_ptr* result) { + std::string query = "SELECT DISTINCT type as table_type FROM sqlite_master"; + + std::shared_ptr statement; + ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, query, &statement)); + + std::shared_ptr reader; + ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create( + statement, SqlSchema::GetTableTypesSchema(), &reader)); + + *result = std::unique_ptr(new RecordBatchStream(reader)); + + return Status::OK(); +} + } // namespace example } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index d41c57a6c93..af5b46db6cb 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -74,6 +74,11 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { Status DoGetTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, std::unique_ptr* result) override; + Status GetFlightInfoTableTypes(const ServerCallContext &context, + const FlightDescriptor &descriptor, + std::unique_ptr *info) override; + Status DoGetTableTypes(const ServerCallContext &context, + std::unique_ptr *result) override; private: sqlite3* db_; diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index dfb700df666..221edee3904 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -290,6 +290,10 @@ std::shared_ptr SqlSchema::GetTablesSchemaWithIncludedSchema() { field("table_schema", binary())}); } +std::shared_ptr SqlSchema::GetTableTypesSchema() { + return arrow::schema({field("table_type", utf8())}); +} + } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 430fe7587ec..25af7eb64ce 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -294,6 +294,10 @@ class SqlSchema { /// flags is set to true. /// \return The default schema template. static std::shared_ptr GetTablesSchemaWithIncludedSchema(); + + /// \brief Gets the Schema used on CommandGetTableTypes response. + /// \return The default schema template. + static std::shared_ptr GetTableTypesSchema(); }; } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index c8cc94ac7c1..2ffbf96c806 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -285,6 +285,23 @@ TEST(TestFlightSqlServer, TestCommandGetSchemas) { ASSERT_EQ(0, table->num_rows()); } +TEST(TestFlightSqlServer, TestCommandGetTableTypes) { + std::unique_ptr flight_info; + ASSERT_OK(sql_client->GetTableTypes({}, &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + DECLARE_ARRAY(table_type, String, ({"table"})); + + const std::shared_ptr
& expected_table = + Table::Make(SqlSchema::GetTableTypesSchema(), {table_type}); + ASSERT_TRUE(expected_table->Equals(*table)); +} + TEST(TestFlightSqlServer, TestCommandStatementUpdate) { int64_t result; ASSERT_OK(sql_client->ExecuteUpdate( From 171d540ac82c3b4bfbb0a5ab53f9d9362dc7c67e Mon Sep 17 00:00:00 2001 From: Jose Almeida <53087160+jcralmeida@users.noreply.github.com> Date: Tue, 12 Oct 2021 14:31:14 -0300 Subject: [PATCH 129/237] [CPP] Implements GetPrimaryKeys on flight sql server (#162) * Add Schema template for the primary keys * Implement GetPrimaryKeys on server * Add an integrated test for GetPrimaryKeys * Fix checkstyle * Add a comment to the query on primary keys query * Use GetFlightInfoForCommand helper method on GetFlightInfoPrimaryKeys Co-authored-by: Rafael Telles --- .../flight-sql/example/sqlite_server.cc | 45 +++++++++++++++++++ .../flight/flight-sql/example/sqlite_server.h | 11 ++++- .../arrow/flight/flight-sql/sql_server.cpp | 6 +++ cpp/src/arrow/flight/flight-sql/sql_server.h | 5 +++ .../flight/flight-sql/sql_server_test.cc | 26 +++++++++++ 5 files changed, 92 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 12e767c9954..e9882bea215 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -311,6 +311,51 @@ Status SQLiteFlightSqlServer::DoGetTableTypes(const ServerCallContext& context, return Status::OK(); } +Status SQLiteFlightSqlServer::GetFlightInfoPrimaryKeys( + const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return GetFlightInfoForCommand(descriptor, info, command, + SqlSchema::GetPrimaryKeysSchema()); +} + +Status +SQLiteFlightSqlServer::DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys &command, + const ServerCallContext &context, + std::unique_ptr *result) { + 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"; + + if (command.has_catalog()) { + table_query << " and catalog_name LIKE '" << command.catalog() << "'"; + } + + if (command.has_schema()) { + table_query << " and schema_name LIKE '" << command.schema() << "'"; + } + + table_query << " and table_name LIKE '" << command.table() << "'"; + + std::shared_ptr statement; + ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, table_query.str(), &statement)); + + std::shared_ptr reader; + ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create( + statement, SqlSchema::GetPrimaryKeysSchema(), &reader)); + + *result = std::unique_ptr( + new RecordBatchStream(reader)); + + return Status::OK(); +} + } // namespace example } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index af5b46db6cb..ea8f167d367 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -80,7 +80,16 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { Status DoGetTableTypes(const ServerCallContext &context, std::unique_ptr *result) override; - private: + Status GetFlightInfoPrimaryKeys(const pb::sql::CommandGetPrimaryKeys &command, + const ServerCallContext &context, + const FlightDescriptor &descriptor, + std::unique_ptr *info) override; + + Status DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys &command, + const ServerCallContext &context, + std::unique_ptr *result) override; + +private: sqlite3* db_; }; diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index 221edee3904..87a22b5806f 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -294,6 +294,12 @@ 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("schema_name", utf8()), + field("table_name", utf8()), field("column_name", utf8()), + field("key_sequence", int64()), field("key_name", utf8())}); +} + } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 25af7eb64ce..515b36eba5c 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -298,6 +298,11 @@ class SqlSchema { /// \brief Gets the Schema used on CommandGetTableTypes response. /// \return The default schema template. static std::shared_ptr GetTableTypesSchema(); + + /// \brief Gets the Schema used on CommandGetPrimaryKeys response when included schema + /// flags is set to true. + /// \return The default schema template. + static std::shared_ptr GetPrimaryKeysSchema(); }; } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 2ffbf96c806..239238135c8 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -323,6 +323,32 @@ TEST(TestFlightSqlServer, TestCommandStatementUpdate) { ASSERT_EQ(3, result); } +TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { + std::unique_ptr flight_info; + std::vector table_types; + ASSERT_OK(sql_client->GetPrimaryKeys({}, nullptr, nullptr, "int%", + &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + DECLARE_NULL_ARRAY(catalog_name, String, 1); + DECLARE_NULL_ARRAY(schema_name, String, 1); + DECLARE_ARRAY(table_name, String, ({"intTable"})); + DECLARE_ARRAY(column_name, String, ({"id"})); + DECLARE_ARRAY(key_sequence, Int64, ({1})); + DECLARE_NULL_ARRAY(key_name, String, 1); + + const std::shared_ptr
& expected_table = Table::Make( + SqlSchema::GetPrimaryKeysSchema(), + {catalog_name, schema_name, table_name, column_name, key_sequence, key_name}); + + ASSERT_TRUE(expected_table->Equals(*table)); +} + auto env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); From 6e2c9e9bf84ccad95cf700a3ac4b94d7636039b7 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 12 Oct 2021 15:27:09 -0300 Subject: [PATCH 130/237] [C++] Implement CommandGetImportedKeys and CommandGetExportedKeys (#163) * Implement CommandGetImportedKeys and CommandGetExportedKeys on Flight SQL Server example * Refactor DoGet methods to reduce code duplication --- .../flight-sql/example/sqlite_server.cc | 111 +++++++++++++++--- .../flight/flight-sql/example/sqlite_server.h | 14 +++ .../arrow/flight/flight-sql/sql_server.cpp | 18 +++ cpp/src/arrow/flight/flight-sql/sql_server.h | 5 + .../flight/flight-sql/sql_server_test.cc | 65 ++++++++++ 5 files changed, 195 insertions(+), 18 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index e9882bea215..c2b97d71540 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -106,6 +106,21 @@ void SQLiteFlightSqlServer::ExecuteSql(const std::string& sql) { } } +Status DoGetSQLiteQuery(sqlite3* db, const std::string& query, + const std::shared_ptr& schema, + std::unique_ptr* result) { + std::shared_ptr statement; + ARROW_RETURN_NOT_OK(SqliteStatement::Create(db, query, &statement)); + + std::shared_ptr reader; + ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create( + statement, schema, &reader)); + + *result = std::unique_ptr(new RecordBatchStream(reader)); + + return Status::OK(); +} + Status GetFlightInfoForCommand(const FlightDescriptor& descriptor, std::unique_ptr* info, const google::protobuf::Message& command, @@ -299,16 +314,7 @@ Status SQLiteFlightSqlServer::DoGetTableTypes(const ServerCallContext& context, std::unique_ptr* result) { std::string query = "SELECT DISTINCT type as table_type FROM sqlite_master"; - std::shared_ptr statement; - ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, query, &statement)); - - std::shared_ptr reader; - ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create( - statement, SqlSchema::GetTableTypesSchema(), &reader)); - - *result = std::unique_ptr(new RecordBatchStream(reader)); - - return Status::OK(); + return DoGetSQLiteQuery(db_, query, SqlSchema::GetTableTypesSchema(), result); } Status SQLiteFlightSqlServer::GetFlightInfoPrimaryKeys( @@ -343,17 +349,86 @@ SQLiteFlightSqlServer::DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys &co table_query << " and table_name LIKE '" << command.table() << "'"; - std::shared_ptr statement; - ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, table_query.str(), &statement)); + return DoGetSQLiteQuery(db_, table_query.str(), SqlSchema::GetPrimaryKeysSchema(), + result); +} - std::shared_ptr reader; - ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create( - statement, SqlSchema::GetPrimaryKeysSchema(), &reader)); +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)"; +} - *result = std::unique_ptr( - new RecordBatchStream(reader)); +Status SQLiteFlightSqlServer::GetFlightInfoImportedKeys( + const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return GetFlightInfoForCommand(descriptor, info, command, + SqlSchema::GetImportedAndExportedKeysSchema()); +} - return Status::OK(); +Status SQLiteFlightSqlServer::DoGetImportedKeys( + const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, + std::unique_ptr* result) { + std::string filter = "fk_table_name = '" + command.table() + "'"; + if (command.has_catalog()) { + filter += " AND fk_catalog_name = '" + command.catalog() + "'"; + } + if (command.has_schema()) { + filter += " AND fk_schema_name = '" + command.schema() + "'"; + } + std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); + + return DoGetSQLiteQuery(db_, query, SqlSchema::GetImportedAndExportedKeysSchema(), + result); +} + +Status SQLiteFlightSqlServer::GetFlightInfoExportedKeys( + const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return GetFlightInfoForCommand(descriptor, info, command, + SqlSchema::GetImportedAndExportedKeysSchema()); +} + +Status SQLiteFlightSqlServer::DoGetExportedKeys( + const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, + std::unique_ptr* result) { + std::string filter = "pk_table_name = '" + command.table() + "'"; + if (command.has_catalog()) { + filter += " AND pk_catalog_name = '" + command.catalog() + "'"; + } + if (command.has_schema()) { + filter += " AND pk_schema_name = '" + command.schema() + "'"; + } + std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); + + return DoGetSQLiteQuery(db_, query, SqlSchema::GetImportedAndExportedKeysSchema(), + result); } } // namespace example diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index ea8f167d367..34c838c07e5 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -79,6 +79,20 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { std::unique_ptr *info) override; Status DoGetTableTypes(const ServerCallContext &context, std::unique_ptr *result) override; + Status GetFlightInfoImportedKeys(const pb::sql::CommandGetImportedKeys &command, + const ServerCallContext &context, + const FlightDescriptor &descriptor, + std::unique_ptr *info) override; + Status DoGetImportedKeys(const pb::sql::CommandGetImportedKeys &command, + const ServerCallContext &context, + std::unique_ptr *result) override; + Status GetFlightInfoExportedKeys(const pb::sql::CommandGetExportedKeys &command, + const ServerCallContext &context, + const FlightDescriptor &descriptor, + std::unique_ptr *info) override; + Status DoGetExportedKeys(const pb::sql::CommandGetExportedKeys &command, + const ServerCallContext &context, + std::unique_ptr *result) override; Status GetFlightInfoPrimaryKeys(const pb::sql::CommandGetPrimaryKeys &command, const ServerCallContext &context, diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index 87a22b5806f..8c4295f098a 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -300,6 +300,24 @@ return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8() field("key_sequence", int64()), field("key_name", utf8())}); } +std::shared_ptr SqlSchema::GetImportedAndExportedKeysSchema() { + return arrow::schema({ + field("pk_catalog_name", utf8(), true), + field("pk_schema_name", utf8(), true), + field("pk_table_name", utf8(), false), + field("pk_column_name", utf8(), false), + field("fk_catalog_name", utf8(), true), + field("fk_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) + }); +} + } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 515b36eba5c..3d2ed36082a 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -303,6 +303,11 @@ class SqlSchema { /// flags is set to true. /// \return The default schema template. static std::shared_ptr GetPrimaryKeysSchema(); + + /// \brief Gets the Schema used on CommandGetImportedKeys and CommandGetExportedKeys + /// response. + /// \return The default schema template. + static std::shared_ptr GetImportedAndExportedKeysSchema(); }; } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 239238135c8..283e1833747 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -349,6 +349,71 @@ TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { ASSERT_TRUE(expected_table->Equals(*table)); } +TEST(TestFlightSqlServer, TestCommandGetImportedKeys) { + std::unique_ptr flight_info; + ASSERT_OK(sql_client->GetImportedKeys({}, NULLPTR, NULLPTR, "intTable", &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + DECLARE_NULL_ARRAY(pk_catalog_name, String, 1); + DECLARE_NULL_ARRAY(pk_schema_name, String, 1); + DECLARE_ARRAY(pk_table_name, String, ({"foreignTable"})); + DECLARE_ARRAY(pk_column_name, String, ({"id"})); + DECLARE_NULL_ARRAY(fk_catalog_name, String, 1); + DECLARE_NULL_ARRAY(fk_schema_name, String, 1); + DECLARE_ARRAY(fk_table_name, String, ({"intTable"})); + DECLARE_ARRAY(fk_column_name, String, ({"foreignId"})); + DECLARE_ARRAY(key_sequence, Int32, ({0})); + DECLARE_NULL_ARRAY(fk_key_name, String, 1); + DECLARE_NULL_ARRAY(pk_key_name, String, 1); + DECLARE_ARRAY(update_rule, UInt8, ({3})); + DECLARE_ARRAY(delete_rule, UInt8, ({3})); + + const std::shared_ptr
& expected_table = + Table::Make(SqlSchema::GetImportedAndExportedKeysSchema(), + {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}); + ASSERT_TRUE(expected_table->Equals(*table)); +} + +TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { + std::unique_ptr flight_info; + ASSERT_OK( + sql_client->GetExportedKeys({}, NULLPTR, NULLPTR, "foreignTable", &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + DECLARE_NULL_ARRAY(pk_catalog_name, String, 1); + DECLARE_NULL_ARRAY(pk_schema_name, String, 1); + DECLARE_ARRAY(pk_table_name, String, ({"foreignTable"})); + DECLARE_ARRAY(pk_column_name, String, ({"id"})); + DECLARE_NULL_ARRAY(fk_catalog_name, String, 1); + DECLARE_NULL_ARRAY(fk_schema_name, String, 1); + DECLARE_ARRAY(fk_table_name, String, ({"intTable"})); + DECLARE_ARRAY(fk_column_name, String, ({"foreignId"})); + DECLARE_ARRAY(key_sequence, Int32, ({0})); + DECLARE_NULL_ARRAY(fk_key_name, String, 1); + DECLARE_NULL_ARRAY(pk_key_name, String, 1); + DECLARE_ARRAY(update_rule, UInt8, ({3})); + DECLARE_ARRAY(delete_rule, UInt8, ({3})); + + const std::shared_ptr
& expected_table = + Table::Make(SqlSchema::GetImportedAndExportedKeysSchema(), + {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}); + ASSERT_TRUE(expected_table->Equals(*table)); +} + auto env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); From 1e437aa119dd8d807918ba46b7b8ec8076142a35 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 5 Oct 2021 14:00:35 -0300 Subject: [PATCH 131/237] Add Create/Close prepared statement actions to sql_server --- .../arrow/flight/flight-sql/sql_server.cpp | 45 +++++++++++++++++++ cpp/src/arrow/flight/flight-sql/sql_server.h | 29 ++++++++++++ 2 files changed, 74 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index 8c4295f098a..a9b49c2432e 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -18,6 +18,7 @@ // Interfaces to use for defining Flight RPC servers. API should be considered // experimental for now +#include "arrow/buffer.h" #include "arrow/flight/flight-sql/sql_server.h" namespace arrow { @@ -126,6 +127,38 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket return Status::Invalid("The defined request is invalid."); } +Status FlightSqlServerBase::ListActions(const ServerCallContext& context, + std::vector* actions) { + *actions = { + FlightSqlServerBase::FLIGHT_SQL_CREATE_PREPARED_STATEMENT, + FlightSqlServerBase::FLIGHT_SQL_CLOSE_PREPARED_STATEMENT + }; + return Status::OK(); +} + +Status FlightSqlServerBase::DoAction(const ServerCallContext &context, + const Action &action, + std::unique_ptr *result) { + if (action.type == FlightSqlServerBase::FLIGHT_SQL_CREATE_PREPARED_STATEMENT.type) { + google::protobuf::Any anyCommand; + anyCommand.ParseFromArray(action.body->data(), static_cast(action.body->size())); + + pb::sql::ActionCreatePreparedStatementRequest command; + anyCommand.UnpackTo(&command); + + return CreatePreparedStatement(command, context, result); + } else if (action.type == FlightSqlServerBase::FLIGHT_SQL_CLOSE_PREPARED_STATEMENT.type) { + google::protobuf::Any anyCommand; + anyCommand.ParseFromArray(action.body->data(), static_cast(action.body->size())); + + pb::sql::ActionClosePreparedStatementRequest command; + anyCommand.UnpackTo(&command); + + return ClosePreparedStatement(command, context, result); + } + return Status::Invalid("The defined request is invalid."); +} + Status FlightSqlServerBase::DoPut(const ServerCallContext& context, std::unique_ptr reader, std::unique_ptr writer) { @@ -263,6 +296,18 @@ Status FlightSqlServerBase::GetFlightInfoImportedKeys( return Status::NotImplemented("DoGetExportedKeys not implemented"); } +Status FlightSqlServerBase::CreatePreparedStatement( + const pb::sql::ActionCreatePreparedStatementRequest& request, + const ServerCallContext& context, std::unique_ptr* p_ptr) { + return Status::NotImplemented("CreatePreparedStatement not implemented"); +} + +Status FlightSqlServerBase::ClosePreparedStatement( + const pb::sql::ActionClosePreparedStatementRequest& request, + const ServerCallContext& context, std::unique_ptr* p_ptr) { + return Status::NotImplemented("ClosePreparedStatement not implemented"); +} + Status FlightSqlServerBase::DoPutCommandStatementUpdate( const pb::sql::CommandStatementUpdate& command, const ServerCallContext& context, std::unique_ptr& reader, diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 3d2ed36082a..0b8683593e1 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -43,6 +43,26 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr reader, std::unique_ptr writer) override; + const ActionType FLIGHT_SQL_CREATE_PREPARED_STATEMENT = + ActionType{.type = "CreatePreparedStatement", + .description = + "Creates a reusable prepared statement resource on the server.\n" + "Request Message: ActionCreatePreparedStatementRequest\n" + "Response Message: ActionCreatePreparedStatementResult"}; + const ActionType FLIGHT_SQL_CLOSE_PREPARED_STATEMENT = ActionType { + .type = "ClosePreparedStatement", + .description = + "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 Gets a FlightInfo for executing a SQL query. /// \param[in] command The CommandStatementQuery object containing the SQL /// statement. @@ -272,6 +292,15 @@ class FlightSqlServerBase : public FlightServerBase { const pb::sql::CommandStatementUpdate& command, const ServerCallContext& context, std::unique_ptr& reader, std::unique_ptr& writer); + + virtual Status CreatePreparedStatement( + const pb::sql::ActionCreatePreparedStatementRequest& request, + const ServerCallContext& context, std::unique_ptr* p_ptr); + + virtual Status ClosePreparedStatement( + const pb::sql::ActionClosePreparedStatementRequest& request, + const ServerCallContext& context, std::unique_ptr* p_ptr); + }; /// \brief Auxiliary class containing all Schemas used on Flight SQL. From 1bf1ea0dd737c89be10646be2a99b6bd08c3106f Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 5 Oct 2021 15:15:03 -0300 Subject: [PATCH 132/237] WIP: Implement prepared statement on server example --- .../flight-sql/example/sqlite_server.cc | 88 +++++++++++++++++++ .../flight/flight-sql/example/sqlite_server.h | 18 ++++ .../flight-sql/example/sqlite_statement.cc | 42 ++++++++- .../arrow/flight/flight-sql/sql_server.cpp | 18 ++-- cpp/src/arrow/flight/flight-sql/sql_server.h | 26 +++--- 5 files changed, 168 insertions(+), 24 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index c2b97d71540..4db140ba997 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -16,6 +16,10 @@ // under the License. #include + +#include +#include +#include #include #include "arrow/api.h" @@ -302,6 +306,90 @@ Status SQLiteFlightSqlServer::DoPutCommandStatementUpdate( return Status::OK(); } +Status SQLiteFlightSqlServer::CreatePreparedStatement( + const pb::sql::ActionCreatePreparedStatementRequest& request, + const ServerCallContext& context, std::unique_ptr* result) { + std::shared_ptr statement; + ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, request.query(), &statement)); + + boost::uuids::uuid uuid = uuid_generator_(); + prepared_statements_[uuid] = statement; + + std::shared_ptr schema; + ARROW_RETURN_NOT_OK(statement->GetSchema(&schema)); + ARROW_ASSIGN_OR_RAISE(auto serialized_schema, ipc::SerializeSchema(*schema)); + + pb::sql::ActionCreatePreparedStatementResult result2; + result2.set_dataset_schema(serialized_schema->ToString()); + result2.set_parameter_schema(""); + result2.set_prepared_statement_handle(boost::uuids::to_string(uuid)); + + google::protobuf::Any any; + any.PackFrom(result2); + + auto buf = Buffer::FromString(any.SerializeAsString()); + *result = std::unique_ptr(new SimpleResultStream({Result{buf}})); + + return Status::OK(); +} + +Status SQLiteFlightSqlServer::ClosePreparedStatement( + const pb::sql::ActionClosePreparedStatementRequest& request, + const ServerCallContext& context, std::unique_ptr* result) { + const std::string& prepared_statement_handle = request.prepared_statement_handle(); + const auto& uuid = boost::lexical_cast(prepared_statement_handle); + + auto search = prepared_statements_.find(uuid); + if (search != prepared_statements_.end()) { + prepared_statements_.erase(uuid); + } else { + return Status::Invalid("Prepared statement not found"); + } + + return Status::OK(); +} + +Status SQLiteFlightSqlServer::GetFlightInfoPreparedStatement( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, const FlightDescriptor& descriptor, + std::unique_ptr* info) { + const std::string& prepared_statement_handle = command.prepared_statement_handle(); + const auto& uuid = boost::lexical_cast(prepared_statement_handle); + + auto search = prepared_statements_.find(uuid); + if (search == prepared_statements_.end()) { + return Status::Invalid("Prepared statement not found"); + } + + std::shared_ptr statement = search->second; + + std::shared_ptr schema; + ARROW_RETURN_NOT_OK(statement->GetSchema(&schema)); + + return GetFlightInfoForCommand(descriptor, info, command, schema); +} + +Status SQLiteFlightSqlServer::DoGetPreparedStatement( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, std::unique_ptr* result) { + const std::string& prepared_statement_handle = command.prepared_statement_handle(); + const auto& uuid = boost::lexical_cast(prepared_statement_handle); + + auto search = prepared_statements_.find(uuid); + if (search == prepared_statements_.end()) { + return Status::Invalid("Prepared statement not found"); + } + + std::shared_ptr statement = search->second; + + std::shared_ptr reader; + ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create(statement, &reader)); + + *result = std::unique_ptr(new RecordBatchStream(reader)); + + return Status::OK(); +} + Status SQLiteFlightSqlServer::GetFlightInfoTableTypes(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index 34c838c07e5..aefe5700767 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -19,6 +19,9 @@ #include +#include +#include + #include "arrow/api.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" @@ -65,6 +68,19 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { const pb::sql::CommandStatementUpdate& update, const ServerCallContext& context, std::unique_ptr& reader, std::unique_ptr& writer) override; + Status CreatePreparedStatement( + const pb::sql::ActionCreatePreparedStatementRequest& request, + const ServerCallContext& context, std::unique_ptr* result) override; + Status ClosePreparedStatement( + const pb::sql::ActionClosePreparedStatementRequest& request, + const ServerCallContext& context, std::unique_ptr* result) override; + Status GetFlightInfoPreparedStatement( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, const FlightDescriptor& descriptor, + std::unique_ptr* info) override; + Status DoGetPreparedStatement(const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, + std::unique_ptr* result) override; Status GetFlightInfoTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, @@ -105,6 +121,8 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { private: sqlite3* db_; + boost::uuids::random_generator uuid_generator_; + std::map> prepared_statements_; }; } // namespace example diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index a024371d624..b8e098c6bde 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -19,6 +19,8 @@ #include +#include + #include "arrow/api.h" namespace arrow { @@ -57,14 +59,52 @@ Status SqliteStatement::Create(sqlite3* db, const std::string& sql, return Status::OK(); } +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 { + return null(); + } +} + Status SqliteStatement::GetSchema(std::shared_ptr* schema) 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); - const int column_type = sqlite3_column_type(stmt_, i); + // SQLite not always provide column types, specially 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. + 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); + } + } + fields.push_back(arrow::field(column_name, data_type)); } diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index a9b49c2432e..5ffa2a8b1e7 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -18,9 +18,10 @@ // Interfaces to use for defining Flight RPC servers. API should be considered // experimental for now -#include "arrow/buffer.h" #include "arrow/flight/flight-sql/sql_server.h" +#include "arrow/buffer.h" + namespace arrow { namespace flight { namespace sql { @@ -129,16 +130,14 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket Status FlightSqlServerBase::ListActions(const ServerCallContext& context, std::vector* actions) { - *actions = { - FlightSqlServerBase::FLIGHT_SQL_CREATE_PREPARED_STATEMENT, - FlightSqlServerBase::FLIGHT_SQL_CLOSE_PREPARED_STATEMENT - }; + *actions = {FlightSqlServerBase::FLIGHT_SQL_CREATE_PREPARED_STATEMENT, + FlightSqlServerBase::FLIGHT_SQL_CLOSE_PREPARED_STATEMENT}; return Status::OK(); } -Status FlightSqlServerBase::DoAction(const ServerCallContext &context, - const Action &action, - std::unique_ptr *result) { +Status FlightSqlServerBase::DoAction(const ServerCallContext& context, + const Action& action, + std::unique_ptr* result) { if (action.type == FlightSqlServerBase::FLIGHT_SQL_CREATE_PREPARED_STATEMENT.type) { google::protobuf::Any anyCommand; anyCommand.ParseFromArray(action.body->data(), static_cast(action.body->size())); @@ -147,7 +146,8 @@ Status FlightSqlServerBase::DoAction(const ServerCallContext &context, anyCommand.UnpackTo(&command); return CreatePreparedStatement(command, context, result); - } else if (action.type == FlightSqlServerBase::FLIGHT_SQL_CLOSE_PREPARED_STATEMENT.type) { + } else if (action.type == + FlightSqlServerBase::FLIGHT_SQL_CLOSE_PREPARED_STATEMENT.type) { google::protobuf::Any anyCommand; anyCommand.ParseFromArray(action.body->data(), static_cast(action.body->size())); diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index 0b8683593e1..d2352147397 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -49,19 +49,18 @@ class FlightSqlServerBase : public FlightServerBase { "Creates a reusable prepared statement resource on the server.\n" "Request Message: ActionCreatePreparedStatementRequest\n" "Response Message: ActionCreatePreparedStatementResult"}; - const ActionType FLIGHT_SQL_CLOSE_PREPARED_STATEMENT = ActionType { - .type = "ClosePreparedStatement", - .description = - "Closes a reusable prepared statement resource on the server.\n" - "Request Message: ActionClosePreparedStatementRequest\n" - "Response Message: N/A"}; + const ActionType FLIGHT_SQL_CLOSE_PREPARED_STATEMENT = + ActionType{.type = "ClosePreparedStatement", + .description = + "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 ListActions(const ServerCallContext& context, + std::vector* actions) override; - Status DoAction(const ServerCallContext &context, - const Action &action, - std::unique_ptr *result) override; + Status DoAction(const ServerCallContext& context, const Action& action, + std::unique_ptr* result) override; /// \brief Gets a FlightInfo for executing a SQL query. /// \param[in] command The CommandStatementQuery object containing the SQL @@ -295,12 +294,11 @@ class FlightSqlServerBase : public FlightServerBase { virtual Status CreatePreparedStatement( const pb::sql::ActionCreatePreparedStatementRequest& request, - const ServerCallContext& context, std::unique_ptr* p_ptr); + const ServerCallContext& context, std::unique_ptr* result); virtual Status ClosePreparedStatement( const pb::sql::ActionClosePreparedStatementRequest& request, - const ServerCallContext& context, std::unique_ptr* p_ptr); - + const ServerCallContext& context, std::unique_ptr* result); }; /// \brief Auxiliary class containing all Schemas used on Flight SQL. From 66a0e41244bc3aa2df7a935dfcd2cc735d3bb138 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 7 Oct 2021 13:35:55 -0300 Subject: [PATCH 133/237] Implement parameter binding on Flight SQL server and example --- .../flight-sql/example/sqlite_server.cc | 108 ++++++++++++++++-- .../flight/flight-sql/example/sqlite_server.h | 4 + .../arrow/flight/flight-sql/sql_server.cpp | 45 +++++--- cpp/src/arrow/flight/flight-sql/sql_server.h | 5 + 4 files changed, 137 insertions(+), 25 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 4db140ba997..68c219ea53c 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -315,17 +315,41 @@ Status SQLiteFlightSqlServer::CreatePreparedStatement( boost::uuids::uuid uuid = uuid_generator_(); prepared_statements_[uuid] = statement; - std::shared_ptr schema; - ARROW_RETURN_NOT_OK(statement->GetSchema(&schema)); - ARROW_ASSIGN_OR_RAISE(auto serialized_schema, ipc::SerializeSchema(*schema)); + std::shared_ptr dataset_schema; + ARROW_RETURN_NOT_OK(statement->GetSchema(&dataset_schema)); + ARROW_ASSIGN_OR_RAISE(auto serialized_dataset_schema, + ipc::SerializeSchema(*dataset_schema)); + + sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); + const int parameter_count = sqlite3_bind_parameter_count(stmt); + std::vector> parameter_fields; + 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({ + field("string", utf8()), + field("bytes", binary()), + field("bigint", int64()), + field("double", float64()), + }))); + } + + const std::shared_ptr& parameter_schema = arrow::schema(parameter_fields); + ARROW_ASSIGN_OR_RAISE(auto serialized_parameter_schema, + ipc::SerializeSchema(*parameter_schema)); - pb::sql::ActionCreatePreparedStatementResult result2; - result2.set_dataset_schema(serialized_schema->ToString()); - result2.set_parameter_schema(""); - result2.set_prepared_statement_handle(boost::uuids::to_string(uuid)); + pb::sql::ActionCreatePreparedStatementResult action_result; + action_result.set_dataset_schema(serialized_dataset_schema->ToString()); + action_result.set_parameter_schema(serialized_parameter_schema->ToString()); + action_result.set_prepared_statement_handle(boost::uuids::to_string(uuid)); google::protobuf::Any any; - any.PackFrom(result2); + any.PackFrom(action_result); auto buf = Buffer::FromString(any.SerializeAsString()); *result = std::unique_ptr(new SimpleResultStream({Result{buf}})); @@ -346,6 +370,7 @@ Status SQLiteFlightSqlServer::ClosePreparedStatement( return Status::Invalid("Prepared statement not found"); } + *result = std::unique_ptr(new SimpleResultStream({})); return Status::OK(); } @@ -390,6 +415,73 @@ Status SQLiteFlightSqlServer::DoGetPreparedStatement( return Status::OK(); } +Status SQLiteFlightSqlServer::DoPutPreparedStatement( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, std::unique_ptr& reader, + std::unique_ptr& writer) { + const std::string& prepared_statement_handle = command.prepared_statement_handle(); + const auto& uuid = boost::lexical_cast(prepared_statement_handle); + + auto search = prepared_statements_.find(uuid); + if (search == prepared_statements_.end()) { + return Status::Invalid("Prepared statement not found"); + } + + std::shared_ptr statement = search->second; + + sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); + + 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 = dynamic_cast(*scalar).value; + + switch (holder->type->id()) { + case Type::INT64: { + int64_t value = dynamic_cast(*holder).value; + sqlite3_bind_int64(stmt, c + 1, value); + break; + } + case Type::FLOAT: { + double value = dynamic_cast(*holder).value; + sqlite3_bind_double(stmt, c + 1, value); + break; + } + case Type::STRING: { + std::shared_ptr buffer = dynamic_cast(*holder).value; + const std::string string = buffer->ToString(); + const char* value = string.c_str(); + sqlite3_bind_text(stmt, c + 1, value, static_cast(strlen(value)), + SQLITE_TRANSIENT); + break; + } + case Type::BINARY: { + std::shared_ptr buffer = dynamic_cast(*holder).value; + sqlite3_bind_blob(stmt, c + 1, buffer->data(), + static_cast(buffer->size()), SQLITE_TRANSIENT); + break; + } + default: + break; + } + } + } + } + + return Status::OK(); +} + Status SQLiteFlightSqlServer::GetFlightInfoTableTypes(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index aefe5700767..c21a664b3c6 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -81,6 +81,10 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { Status DoGetPreparedStatement(const pb::sql::CommandPreparedStatementQuery& command, const ServerCallContext& context, std::unique_ptr* result) override; + Status DoPutPreparedStatement(const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext &context, + std::unique_ptr& reader, + std::unique_ptr& writer) override; Status GetFlightInfoTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index 5ffa2a8b1e7..8f124b43da2 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -128,6 +128,28 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket 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; + any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size())); + + if (any.Is()) { + pb::sql::CommandStatementUpdate command; + any.UnpackTo(&command); + return DoPutCommandStatementUpdate(command, context, reader, writer); + } else if (any.Is()) { + pb::sql::CommandPreparedStatementQuery command; + any.UnpackTo(&command); + return DoPutPreparedStatement(command, context, reader, writer); + } + + return Status::Invalid("The defined request is invalid."); +} + Status FlightSqlServerBase::ListActions(const ServerCallContext& context, std::vector* actions) { *actions = {FlightSqlServerBase::FLIGHT_SQL_CREATE_PREPARED_STATEMENT, @@ -159,23 +181,6 @@ Status FlightSqlServerBase::DoAction(const ServerCallContext& context, 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; - any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size())); - - if (any.Is()) { - pb::sql::CommandStatementUpdate command; - any.UnpackTo(&command); - return DoPutCommandStatementUpdate(command, context, reader, writer); - } - - return Status::Invalid("The defined request is invalid."); -} - Status FlightSqlServerBase::GetFlightInfoCatalogs(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { @@ -307,6 +312,12 @@ Status FlightSqlServerBase::ClosePreparedStatement( const ServerCallContext& context, std::unique_ptr* p_ptr) { return Status::NotImplemented("ClosePreparedStatement not implemented"); } +Status FlightSqlServerBase::DoPutPreparedStatement(const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext &context, + std::unique_ptr &reader, + std::unique_ptr &writer) { + return Status::NotImplemented("DoPutPreparedStatement not implemented"); +} Status FlightSqlServerBase::DoPutCommandStatementUpdate( const pb::sql::CommandStatementUpdate& command, const ServerCallContext& context, diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index d2352147397..aaa233df7ae 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -299,6 +299,11 @@ class FlightSqlServerBase : public FlightServerBase { virtual Status ClosePreparedStatement( const pb::sql::ActionClosePreparedStatementRequest& request, const ServerCallContext& context, std::unique_ptr* result); + + virtual Status DoPutPreparedStatement(const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext &context, + std::unique_ptr& reader, + std::unique_ptr& writer); }; /// \brief Auxiliary class containing all Schemas used on Flight SQL. From b8417b1a2295730772de68975b7a82c0e1bb655d Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 8 Oct 2021 13:55:19 -0300 Subject: [PATCH 134/237] Remove GetArrowType method duplicate --- .../flight-sql/example/sqlite_server.cc | 22 +++++++++++++++++++ .../flight/flight-sql/example/sqlite_server.h | 2 ++ .../flight-sql/example/sqlite_statement.cc | 22 +------------------ .../sqlite_tables_schema_batch_reader.cpp | 20 +++-------------- .../sqlite_tables_schema_batch_reader.h | 8 +++---- 5 files changed, 31 insertions(+), 43 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 68c219ea53c..b915cc0fc4d 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -17,6 +17,7 @@ #include +#include #include #include #include @@ -34,6 +35,27 @@ namespace flight { namespace sql { namespace example { +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 { + return null(); + } +} + std::string PrepareQueryForGetTables(const pb::sql::CommandGetTables& command) { std::stringstream table_query; diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index c21a664b3c6..68901b339d8 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -32,6 +32,8 @@ namespace flight { namespace sql { namespace example { +std::shared_ptr GetArrowType(const char* sqlite_type); + /// \brief Example implementation of FlightSqlServerBase backed by an in-memory SQLite3 /// database. class SQLiteFlightSqlServer : public FlightSqlServerBase { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index b8e098c6bde..4aeaf80be85 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -22,6 +22,7 @@ #include #include "arrow/api.h" +#include "arrow/flight/flight-sql/example/sqlite_server.h" namespace arrow { namespace flight { @@ -59,27 +60,6 @@ Status SqliteStatement::Create(sqlite3* db, const std::string& sql, return Status::OK(); } -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 { - return null(); - } -} - Status SqliteStatement::GetSchema(std::shared_ptr* schema) const { std::vector> fields; int column_count = sqlite3_column_count(stmt_); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp index 459fe34f2fb..4a20b5de308 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp @@ -22,14 +22,15 @@ #include #include -#include #include +#include "arrow/flight/flight-sql/example/sqlite_server.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" namespace arrow { namespace flight { namespace sql { +namespace example { std::shared_ptr SqliteTablesWithSchemaBatchReader::schema() const { return SqlSchema::GetTablesSchemaWithIncludedSchema(); @@ -99,22 +100,7 @@ Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* return Status::OK(); } -std::shared_ptr SqliteTablesWithSchemaBatchReader::GetArrowType( - const char* sqlite_type) { - 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 { - return null(); - } -} +} // namespace example } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h index d44f7d53e58..66a1038e833 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h @@ -30,6 +30,7 @@ namespace arrow { namespace flight { namespace sql { +namespace example { class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { private: @@ -49,12 +50,9 @@ class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { std::shared_ptr schema() const override; Status ReadNext(std::shared_ptr* batch) override; - - /// Convert a column type to a ArrowType. - /// \param sqlite_type the sqlite type. - /// \return The equivalent ArrowType. - static std::shared_ptr GetArrowType(const char* sqlite_type); }; + +} // namespace example } // namespace sql } // namespace flight } // namespace arrow From 54ecc387d745b38b66f1bd6a1d0d96cc05cd9ffa Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 8 Oct 2021 14:04:05 -0300 Subject: [PATCH 135/237] Add missing docs for GetArrowType --- cpp/src/arrow/flight/flight-sql/example/sqlite_server.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index 68901b339d8..cb179be9fd5 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -32,6 +32,9 @@ namespace flight { namespace sql { namespace example { +/// 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 Example implementation of FlightSqlServerBase backed by an in-memory SQLite3 From a9c11db01101843e96e8c88468959bd9b8e21e4f Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 8 Oct 2021 15:07:01 -0300 Subject: [PATCH 136/237] Add integration tests for PreparedStatement query --- .../flight-sql/example/sqlite_server.cc | 20 ++-- .../arrow/flight/flight-sql/sql_server.cpp | 1 + .../flight/flight-sql/sql_server_test.cc | 95 +++++++++++++++++++ 3 files changed, 108 insertions(+), 8 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index b915cc0fc4d..b0544d1a9a0 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -15,12 +15,11 @@ // specific language governing permissions and limitations // under the License. -#include - #include #include #include #include +#include #include #include "arrow/api.h" @@ -353,12 +352,17 @@ Status SQLiteFlightSqlServer::CreatePreparedStatement( } else { parameter_name = parameter_name_chars; } - parameter_fields.push_back(field(parameter_name, dense_union({ - field("string", utf8()), - field("bytes", binary()), - field("bigint", int64()), - field("double", float64()), - }))); + + // 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 = dense_union({ + field("string", utf8()), + field("bytes", binary()), + field("bigint", int64()), + field("double", float64()), + }); + parameter_fields.push_back(field(parameter_name, dense_union_type)); } const std::shared_ptr& parameter_schema = arrow::schema(parameter_fields); diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index 8f124b43da2..57c4963a71e 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -312,6 +312,7 @@ Status FlightSqlServerBase::ClosePreparedStatement( const ServerCallContext& context, std::unique_ptr* p_ptr) { return Status::NotImplemented("ClosePreparedStatement not implemented"); } + Status FlightSqlServerBase::DoPutPreparedStatement(const pb::sql::CommandPreparedStatementQuery& command, const ServerCallContext &context, std::unique_ptr &reader, diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 283e1833747..6aa90444743 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -323,6 +323,101 @@ TEST(TestFlightSqlServer, TestCommandStatementUpdate) { ASSERT_EQ(3, result); } +TEST(TestFlightSqlServer, TestCommandPreparedStatementQuery) { + std::shared_ptr prepared_statement; + ASSERT_OK(sql_client->Prepare({}, "SELECT * FROM intTable", &prepared_statement)); + + std::unique_ptr flight_info; + ASSERT_OK(prepared_statement->Execute(&flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + 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())}); + + DECLARE_ARRAY(id_array, Int64, ({1, 2, 3})); + DECLARE_ARRAY(keyname_array, String, ({"one", "zero", "negative one"})); + DECLARE_ARRAY(value_array, Int64, ({1, 0, -1})); + DECLARE_ARRAY(foreignId_array, Int64, ({1, 1, 1})); + + const std::shared_ptr
& expected_table = Table::Make( + expected_schema, {id_array, keyname_array, value_array, foreignId_array}); + + ASSERT_TRUE(expected_table->Equals(*table)); +} + +TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) { + std::shared_ptr prepared_statement; + ASSERT_OK(sql_client->Prepare({}, "SELECT * FROM intTable WHERE keyName LIKE ?", + &prepared_statement)); + + std::shared_ptr parameter_schema; + ASSERT_OK(prepared_statement->GetParameterSchema(¶meter_schema)); + + const std::shared_ptr& expected_parameter_schema = + arrow::schema({arrow::field("parameter_1", dense_union({ + field("string", utf8()), + field("bytes", binary()), + field("bigint", int64()), + field("double", float64()), + }))}); + + ASSERT_TRUE(expected_parameter_schema->Equals(*parameter_schema)); + + std::shared_ptr type_ids; + ArrayFromVector({0}, &type_ids); + std::shared_ptr offsets; + ArrayFromVector({0}, &offsets); + + std::shared_ptr string_array; + ArrayFromVector({"%one"}, &string_array); + std::shared_ptr bytes_array; + ArrayFromVector({}, &bytes_array); + std::shared_ptr bigint_array; + ArrayFromVector({}, &bigint_array); + std::shared_ptr double_array; + ArrayFromVector({}, &double_array); + + 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)); + + std::unique_ptr flight_info; + ASSERT_OK(prepared_statement->Execute(&flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + 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())}); + + DECLARE_ARRAY(id_array, Int64, ({1, 3})); + DECLARE_ARRAY(keyname_array, String, ({"one", "negative one"})); + DECLARE_ARRAY(value_array, Int64, ({1, -1})); + DECLARE_ARRAY(foreignId_array, Int64, ({1, 1})); + + const std::shared_ptr
& expected_table = Table::Make( + expected_schema, {id_array, keyname_array, value_array, foreignId_array}); + + ASSERT_TRUE(expected_table->Equals(*table)); +} + TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { std::unique_ptr flight_info; std::vector table_types; From 247be5bc7b0fdef71d4ce16747fc1185b59bb245 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 8 Oct 2021 18:04:35 -0300 Subject: [PATCH 137/237] Make changes regarding to reviews --- .../flight-sql/example/sqlite_server.cc | 32 +++++++++---------- .../flight/flight-sql/example/sqlite_server.h | 13 +++++++- .../flight-sql/example/sqlite_statement.cc | 9 ++++-- cpp/src/arrow/flight/flight-sql/sql_server.h | 24 +++++++++++--- .../flight/flight-sql/sql_server_test.cc | 10 ++---- 5 files changed, 58 insertions(+), 30 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index b0544d1a9a0..e75c4f3e557 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -344,6 +344,13 @@ Status SQLiteFlightSqlServer::CreatePreparedStatement( 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; @@ -352,16 +359,6 @@ Status SQLiteFlightSqlServer::CreatePreparedStatement( } else { parameter_name = parameter_name_chars; } - - // 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 = dense_union({ - field("string", utf8()), - field("bytes", binary()), - field("bigint", int64()), - field("double", float64()), - }); parameter_fields.push_back(field(parameter_name, dense_union_type)); } @@ -396,6 +393,7 @@ Status SQLiteFlightSqlServer::ClosePreparedStatement( return Status::Invalid("Prepared statement not found"); } + // Need to instantiate a ResultStream, otherwise clients can not wait for completion. *result = std::unique_ptr(new SimpleResultStream({})); return Status::OK(); } @@ -457,6 +455,7 @@ Status SQLiteFlightSqlServer::DoPutPreparedStatement( sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); + // Loading parameters received from the RecordBatches on the underlying sqlite3_stmt. FlightStreamChunk chunk; while (true) { RETURN_NOT_OK(reader->Next(&chunk)); @@ -471,21 +470,21 @@ Status SQLiteFlightSqlServer::DoPutPreparedStatement( const std::shared_ptr& column = record_batch->column(c); ARROW_ASSIGN_OR_RAISE(std::shared_ptr scalar, column->GetScalar(i)); - auto& holder = dynamic_cast(*scalar).value; + auto& holder = reinterpret_cast(*scalar).value; switch (holder->type->id()) { case Type::INT64: { - int64_t value = dynamic_cast(*holder).value; + int64_t value = reinterpret_cast(*holder).value; sqlite3_bind_int64(stmt, c + 1, value); break; } case Type::FLOAT: { - double value = dynamic_cast(*holder).value; + double value = reinterpret_cast(*holder).value; sqlite3_bind_double(stmt, c + 1, value); break; } case Type::STRING: { - std::shared_ptr buffer = dynamic_cast(*holder).value; + std::shared_ptr buffer = reinterpret_cast(*holder).value; const std::string string = buffer->ToString(); const char* value = string.c_str(); sqlite3_bind_text(stmt, c + 1, value, static_cast(strlen(value)), @@ -493,13 +492,14 @@ Status SQLiteFlightSqlServer::DoPutPreparedStatement( break; } case Type::BINARY: { - std::shared_ptr buffer = dynamic_cast(*holder).value; + std::shared_ptr buffer = reinterpret_cast(*holder).value; sqlite3_bind_blob(stmt, c + 1, buffer->data(), static_cast(buffer->size()), SQLITE_TRANSIENT); break; } default: - break; + return Status::Invalid("Received unsupported data type: ", + holder->type->ToString()); } } } diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index cb179be9fd5..db446141a39 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -32,11 +32,22 @@ namespace flight { namespace sql { namespace example { -/// Convert a column type to a ArrowType. +/// \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 { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index 4aeaf80be85..da1626516f7 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -66,7 +66,7 @@ Status SqliteStatement::GetSchema(std::shared_ptr* schema) const { for (int i = 0; i < column_count; i++) { const char* column_name = sqlite3_column_name(stmt_, i); - // SQLite not always provide column types, specially when the statement has not been + // 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 @@ -74,7 +74,8 @@ Status SqliteStatement::GetSchema(std::shared_ptr* schema) const { // 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. + // 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) { @@ -82,6 +83,10 @@ Status SqliteStatement::GetSchema(std::shared_ptr* schema) const { 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(); } } diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/sql_server.h index aaa233df7ae..e1adb09b438 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/sql_server.h @@ -292,18 +292,34 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr& reader, std::unique_ptr& writer); + /// \brief Create a prepared statement from given SQL statement. + /// \param[in] request The ActionCreatePreparedStatementRequest object containing the + /// SQL statement. + /// \param[in] context The call context. + /// \param[out] result ResultStream containing a ActionCreatePreparedStatementResult. virtual Status CreatePreparedStatement( const pb::sql::ActionCreatePreparedStatementRequest& request, const ServerCallContext& context, std::unique_ptr* result); + /// \brief Closes a prepared statement. + /// \param[in] request The ActionClosePreparedStatementRequest object containing the + /// prepared statement handle. + /// \param[in] context The call context. + /// \param[out] result Empty ResultStream. virtual Status ClosePreparedStatement( const pb::sql::ActionClosePreparedStatementRequest& request, const ServerCallContext& context, std::unique_ptr* result); - virtual Status DoPutPreparedStatement(const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext &context, - std::unique_ptr& reader, - std::unique_ptr& writer); + /// \brief Binds parameters to given prepared statement. + /// \param[in] command The CommandPreparedStatementQuery object containing the + /// prepared statement handle. + /// \param[in] context The call context. + /// \param[in] reader A sequence of uploaded record batches. + /// \param[in] writer Send metadata back to the client. + virtual Status DoPutPreparedStatement( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, std::unique_ptr& reader, + std::unique_ptr& writer); }; /// \brief Auxiliary class containing all Schemas used on Flight SQL. diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index 6aa90444743..fc3729ae5d8 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -359,13 +360,8 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) std::shared_ptr parameter_schema; ASSERT_OK(prepared_statement->GetParameterSchema(¶meter_schema)); - const std::shared_ptr& expected_parameter_schema = - arrow::schema({arrow::field("parameter_1", dense_union({ - field("string", utf8()), - field("bytes", binary()), - field("bigint", int64()), - field("double", float64()), - }))}); + const std::shared_ptr& expected_parameter_schema = arrow::schema( + {arrow::field("parameter_1", example::GetUnknownColumnDataType())}); ASSERT_TRUE(expected_parameter_schema->Equals(*parameter_schema)); From 22320fd20ecf83e92406df0dd3c3d97d7f465895 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 8 Oct 2021 18:04:35 -0300 Subject: [PATCH 138/237] Make changes regarding to reviews --- cpp/src/arrow/flight/flight-sql/client_impl.h | 35 +++++++++++++++- .../arrow/flight/flight-sql/client_test.cc | 40 +++++++++++++++++++ 2 files changed, 74 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 2f3b104a606..988af0b6d63 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -313,7 +313,40 @@ Status PreparedStatementT::Execute(std::unique_ptr* info) { } template -Status PreparedStatementT::SetParameters(std::shared_ptr parameter_binding_) { +Status PreparedStatementT::ExecuteUpdate(int64_t* rows) { + if (is_closed) { + return Status::Invalid("Statement already closed."); + } + pb::sql::CommandPreparedStatementUpdate command; + command.set_prepared_statement_handle( + prepared_statement_result.prepared_statement_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)); + ARROW_RETURN_NOT_OK(writer->WriteRecordBatch( + *arrow::RecordBatch::Make(schema, 0, (std::vector>){}))); + } + ARROW_RETURN_NOT_OK(writer->DoneWriting()); + std::shared_ptr metadata; + ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata)); + pb::sql::DoPutUpdateResult doPutUpdateResult; + const std::string& string = metadata->ToString(); + doPutUpdateResult.ParseFrom(string); + *rows = doPutUpdateResult.record_count(); + ARROW_RETURN_NOT_OK(writer->Close()); + return Status::OK(); +} + +template +Status PreparedStatementT::SetParameters( + std::shared_ptr parameter_binding_) { parameter_binding = std::move(parameter_binding_); return Status::OK(); diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index eb00eaf5b62..26ac8989938 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -455,6 +455,46 @@ TEST(TestFlightSqlClient, TestGetSqlInfo) { (void) sql_client.GetSqlInfo(call_options, sql_info, &flight_info); } +TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { + auto* client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sql_client(client_mock_ptr); + const std::string query = "SELECT * FROM IRRELEVANT"; + const FlightCallOptions call_options; + int64_t expected_rows = 100L; + pb::sql::DoPutUpdateResult result; + result.set_record_count(expected_rows); + auto buffer = std::make_shared( + reinterpret_cast(result.SerializeAsString().data()), + result.ByteSizeLong()); + ON_CALL(*client_mock, DoAction) + .WillByDefault([&query](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(); + }); + ON_CALL(*client_mock, 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(); + }); + int64_t rows; + std::shared_ptr> prepared_statement; + ASSERT_OK(sql_client.Prepare(call_options, query, &prepared_statement)); + ASSERT_OK(prepared_statement->ExecuteUpdate(&rows)); + EXPECT_EQ(expected_rows, rows); +} + } // namespace sql } // namespace flight } // namespace arrow From 2d23d07476b9be306507a00b76ffd8eae5053b7b Mon Sep 17 00:00:00 2001 From: Abner Eduardo Ferreira Date: Tue, 12 Oct 2021 16:36:22 -0300 Subject: [PATCH 139/237] Fix broken test for PreparedStatement.ExecuteUpdate without parameter binding --- .../arrow/flight/flight-sql/client_test.cc | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 26ac8989938..8c2f671e0d5 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -464,20 +464,21 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { int64_t expected_rows = 100L; pb::sql::DoPutUpdateResult result; result.set_record_count(expected_rows); - auto buffer = std::make_shared( - reinterpret_cast(result.SerializeAsString().data()), - result.ByteSizeLong()); + auto buffer = Buffer::FromString(result.SerializeAsString()); + google::protobuf::Any command; + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + prepared_statement_result.set_prepared_statement_handle(query); + command.PackFrom(prepared_statement_result); + const std::string& serializedCommand = command.SerializeAsString(); ON_CALL(*client_mock, DoAction) - .WillByDefault([&query](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())}})); + .WillByDefault([&serializedCommand](const FlightCallOptions& options, + const Action& action, + std::unique_ptr* results) { + *results = std::unique_ptr( + new SimpleResultStream({Result{Buffer::FromString(serializedCommand)}})); return Status::OK(); }); + EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); ON_CALL(*client_mock, DoPut) .WillByDefault([&buffer](const FlightCallOptions& options, const FlightDescriptor& descriptor1, @@ -488,6 +489,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { writer->reset(new FlightStreamWriterMock()); return Status::OK(); }); + EXPECT_CALL(*client_mock, DoPut(_, _, _, _, _)); int64_t rows; std::shared_ptr> prepared_statement; ASSERT_OK(sql_client.Prepare(call_options, query, &prepared_statement)); From d9ab9348e49ed2aef29183df0f0315693cc6b676 Mon Sep 17 00:00:00 2001 From: Abner Eduardo Ferreira Date: Tue, 12 Oct 2021 17:19:46 -0300 Subject: [PATCH 140/237] Add test case for PreparedStatement.ExecuteUpdate with parameter binding --- .../arrow/flight/flight-sql/client_test.cc | 54 +++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 8c2f671e0d5..af1e2e0fff3 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -497,6 +497,60 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { EXPECT_EQ(expected_rows, rows); } +TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding) { + auto* client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sql_client(client_mock_ptr); + const std::string query = "SELECT * FROM IRRELEVANT"; + const FlightCallOptions call_options; + int64_t expected_rows = 100L; + pb::sql::DoPutUpdateResult result; + result.set_record_count(expected_rows); + auto buffer = Buffer::FromString(result.SerializeAsString()); + google::protobuf::Any command; + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + prepared_statement_result.set_prepared_statement_handle(query); + command.PackFrom(prepared_statement_result); + const std::string& serializedCommand = command.SerializeAsString(); + ON_CALL(*client_mock, DoAction) + .WillByDefault([&serializedCommand](const FlightCallOptions& options, + const Action& action, + std::unique_ptr* results) { + *results = std::unique_ptr( + new SimpleResultStream({Result{Buffer::FromString(serializedCommand)}})); + return Status::OK(); + }); + EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); + ON_CALL(*client_mock, 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(); + }); + int64_t rows; + std::shared_ptr> prepared_statement; + ASSERT_OK(sql_client.Prepare(call_options, query, &prepared_statement)); + const auto& schema = arrow::schema( + {arrow::field("field0", arrow::utf8()), + arrow::field("field1", arrow::uint8())}); + std::shared_ptr stringArray; + std::shared_ptr uInt8Array; + const std::vector stringData {"Loren", "Ipsum", "Foo", "Bar", "Baz"}; + const std::vector uInt8Data {0, 10, 15, 20, 25}; + ArrayFromVector(stringData, &stringArray); + ArrayFromVector(uInt8Data, &uInt8Array); + std::shared_ptr recordBatch = + RecordBatch::Make(schema, 100, {stringArray, uInt8Array}); + EXPECT_CALL(*client_mock, DoPut(_, _, schema, _, _)); + ASSERT_OK(prepared_statement->SetParameters(recordBatch)); + ASSERT_OK(prepared_statement->ExecuteUpdate(&rows)); + EXPECT_EQ(expected_rows, rows); +} + } // namespace sql } // namespace flight } // namespace arrow From 86fe0463d75db9059b4a73a3d71a4a17ccf05905 Mon Sep 17 00:00:00 2001 From: Abner Eduardo Ferreira Date: Tue, 12 Oct 2021 17:33:48 -0300 Subject: [PATCH 141/237] Fix rebase issues --- cpp/src/arrow/flight/flight-sql/client.h | 5 +++++ cpp/src/arrow/flight/flight-sql/client_impl.h | 8 ++++++-- cpp/src/arrow/flight/flight-sql/client_test.cc | 9 +++++++++ 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 9fbc8c8d1bf..4690958c8f1 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -60,6 +60,11 @@ class ARROW_EXPORT PreparedStatementT { /// \return Status. Status Execute(std::unique_ptr* info); + /// \brief Executes the prepared statement update query on the server. + /// \param rows[out] The number of rows affected. + /// \return Status. + Status ExecuteUpdate(int64_t* rows); + /// \brief Retrieve the parameter schema from the query. /// \param schema The parameter schema from the query. /// \return Status. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 988af0b6d63..eb77a4139f5 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -317,12 +317,14 @@ Status PreparedStatementT::ExecuteUpdate(int64_t* rows) { if (is_closed) { return Status::Invalid("Statement already closed."); } + pb::sql::CommandPreparedStatementUpdate command; command.set_prepared_statement_handle( prepared_statement_result.prepared_statement_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)); @@ -333,12 +335,14 @@ Status PreparedStatementT::ExecuteUpdate(int64_t* rows) { ARROW_RETURN_NOT_OK(writer->WriteRecordBatch( *arrow::RecordBatch::Make(schema, 0, (std::vector>){}))); } + ARROW_RETURN_NOT_OK(writer->DoneWriting()); std::shared_ptr metadata; ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata)); pb::sql::DoPutUpdateResult doPutUpdateResult; - const std::string& string = metadata->ToString(); - doPutUpdateResult.ParseFrom(string); + const std::string& metadataAsString = metadata->ToString(); + + doPutUpdateResult.ParseFrom(metadataAsString); *rows = doPutUpdateResult.record_count(); ARROW_RETURN_NOT_OK(writer->Close()); return Status::OK(); diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index af1e2e0fff3..6821a0a6432 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -459,6 +459,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sql_client(client_mock_ptr); + const std::string query = "SELECT * FROM IRRELEVANT"; const FlightCallOptions call_options; int64_t expected_rows = 100L; @@ -470,6 +471,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { prepared_statement_result.set_prepared_statement_handle(query); command.PackFrom(prepared_statement_result); const std::string& serializedCommand = command.SerializeAsString(); + ON_CALL(*client_mock, DoAction) .WillByDefault([&serializedCommand](const FlightCallOptions& options, const Action& action, @@ -479,6 +481,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { return Status::OK(); }); EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); + ON_CALL(*client_mock, DoPut) .WillByDefault([&buffer](const FlightCallOptions& options, const FlightDescriptor& descriptor1, @@ -490,6 +493,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { return Status::OK(); }); EXPECT_CALL(*client_mock, DoPut(_, _, _, _, _)); + int64_t rows; std::shared_ptr> prepared_statement; ASSERT_OK(sql_client.Prepare(call_options, query, &prepared_statement)); @@ -501,6 +505,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sql_client(client_mock_ptr); + const std::string query = "SELECT * FROM IRRELEVANT"; const FlightCallOptions call_options; int64_t expected_rows = 100L; @@ -512,6 +517,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding prepared_statement_result.set_prepared_statement_handle(query); command.PackFrom(prepared_statement_result); const std::string& serializedCommand = command.SerializeAsString(); + ON_CALL(*client_mock, DoAction) .WillByDefault([&serializedCommand](const FlightCallOptions& options, const Action& action, @@ -521,6 +527,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding return Status::OK(); }); EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); + ON_CALL(*client_mock, DoPut) .WillByDefault([&buffer](const FlightCallOptions& options, const FlightDescriptor& descriptor1, @@ -537,6 +544,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding const auto& schema = arrow::schema( {arrow::field("field0", arrow::utf8()), arrow::field("field1", arrow::uint8())}); + std::shared_ptr stringArray; std::shared_ptr uInt8Array; const std::vector stringData {"Loren", "Ipsum", "Foo", "Bar", "Baz"}; @@ -545,6 +553,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding ArrayFromVector(uInt8Data, &uInt8Array); std::shared_ptr recordBatch = RecordBatch::Make(schema, 100, {stringArray, uInt8Array}); + EXPECT_CALL(*client_mock, DoPut(_, _, schema, _, _)); ASSERT_OK(prepared_statement->SetParameters(recordBatch)); ASSERT_OK(prepared_statement->ExecuteUpdate(&rows)); From 464248d4ef4fa8b9f17be8b96f5671f0f9eec2a6 Mon Sep 17 00:00:00 2001 From: Abner Eduardo Ferreira Date: Wed, 13 Oct 2021 12:05:25 -0300 Subject: [PATCH 142/237] WIP: Refactor test cases for PreparedStatement.ExecuteUpdate --- .../arrow/flight/flight-sql/client_test.cc | 92 ++++++------------- 1 file changed, 30 insertions(+), 62 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 6821a0a6432..87d6499115d 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -455,9 +455,10 @@ TEST(TestFlightSqlClient, TestGetSqlInfo) { (void) sql_client.GetSqlInfo(call_options, sql_info, &flight_info); } -TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); +template +inline void AssertTestPreparedStatementExecuteUpdateOk(Func func) { + FlightClientMock client_mock; + std::unique_ptr client_mock_ptr(&client_mock); FlightSqlClientT sql_client(client_mock_ptr); const std::string query = "SELECT * FROM IRRELEVANT"; @@ -472,7 +473,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { command.PackFrom(prepared_statement_result); const std::string& serializedCommand = command.SerializeAsString(); - ON_CALL(*client_mock, DoAction) + ON_CALL(client_mock, DoAction) .WillByDefault([&serializedCommand](const FlightCallOptions& options, const Action& action, std::unique_ptr* results) { @@ -480,9 +481,9 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { new SimpleResultStream({Result{Buffer::FromString(serializedCommand)}})); return Status::OK(); }); - EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); + EXPECT_CALL(client_mock, DoAction(_, _, _)).Times(2); - ON_CALL(*client_mock, DoPut) + ON_CALL(client_mock, DoPut) .WillByDefault([&buffer](const FlightCallOptions& options, const FlightDescriptor& descriptor1, const std::shared_ptr& schema, @@ -492,72 +493,39 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { writer->reset(new FlightStreamWriterMock()); return Status::OK(); }); - EXPECT_CALL(*client_mock, DoPut(_, _, _, _, _)); + EXPECT_CALL(client_mock, DoPut(_, _, _, _, _)); int64_t rows; std::shared_ptr> prepared_statement; ASSERT_OK(sql_client.Prepare(call_options, query, &prepared_statement)); + func(prepared_statement, client_mock); ASSERT_OK(prepared_statement->ExecuteUpdate(&rows)); EXPECT_EQ(expected_rows, rows); } -TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); - - const std::string query = "SELECT * FROM IRRELEVANT"; - const FlightCallOptions call_options; - int64_t expected_rows = 100L; - pb::sql::DoPutUpdateResult result; - result.set_record_count(expected_rows); - auto buffer = Buffer::FromString(result.SerializeAsString()); - google::protobuf::Any command; - pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; - prepared_statement_result.set_prepared_statement_handle(query); - command.PackFrom(prepared_statement_result); - const std::string& serializedCommand = command.SerializeAsString(); - - ON_CALL(*client_mock, DoAction) - .WillByDefault([&serializedCommand](const FlightCallOptions& options, - const Action& action, - std::unique_ptr* results) { - *results = std::unique_ptr( - new SimpleResultStream({Result{Buffer::FromString(serializedCommand)}})); - return Status::OK(); - }); - EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); +TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { + AssertTestPreparedStatementExecuteUpdateOk( + [](const std::shared_ptr>& + prepared_statement, FlightClientMock& client_mock) {}); +} - ON_CALL(*client_mock, 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(); +TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding) { + AssertTestPreparedStatementExecuteUpdateOk( + [](const std::shared_ptr>& + prepared_statement, FlightClientMock& client_mock) { + const auto& schema = arrow::schema({arrow::field("field0", arrow::utf8()), + arrow::field("field1", arrow::uint8())}); + std::shared_ptr stringArray; + std::shared_ptr uInt8Array; + const std::vector stringData {"Loren", "Ipsum", "Foo", "Bar", "Baz"}; + const std::vector uInt8Data {0, 10, 15, 20, 25}; + ArrayFromVector(stringData, &stringArray); + ArrayFromVector(uInt8Data, &uInt8Array); + std::shared_ptr recordBatch = + RecordBatch::Make(schema, 100, {stringArray, uInt8Array}); + ASSERT_OK(prepared_statement->SetParameters(recordBatch)); + EXPECT_CALL(client_mock, DoPut(_, _, schema, _, _)); }); - int64_t rows; - std::shared_ptr> prepared_statement; - ASSERT_OK(sql_client.Prepare(call_options, query, &prepared_statement)); - const auto& schema = arrow::schema( - {arrow::field("field0", arrow::utf8()), - arrow::field("field1", arrow::uint8())}); - - std::shared_ptr stringArray; - std::shared_ptr uInt8Array; - const std::vector stringData {"Loren", "Ipsum", "Foo", "Bar", "Baz"}; - const std::vector uInt8Data {0, 10, 15, 20, 25}; - ArrayFromVector(stringData, &stringArray); - ArrayFromVector(uInt8Data, &uInt8Array); - std::shared_ptr recordBatch = - RecordBatch::Make(schema, 100, {stringArray, uInt8Array}); - - EXPECT_CALL(*client_mock, DoPut(_, _, schema, _, _)); - ASSERT_OK(prepared_statement->SetParameters(recordBatch)); - ASSERT_OK(prepared_statement->ExecuteUpdate(&rows)); - EXPECT_EQ(expected_rows, rows); } } // namespace sql From 14d2725e337e1d297eeb6638742f94c5bf560728 Mon Sep 17 00:00:00 2001 From: Abner Eduardo Ferreira Date: Wed, 13 Oct 2021 14:46:46 -0300 Subject: [PATCH 143/237] WIP: Refactor test cases for PreparedStatement.ExecuteUpdate to use lambda functions --- .../arrow/flight/flight-sql/client_test.cc | 135 ++++++++++-------- 1 file changed, 73 insertions(+), 62 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 87d6499115d..d741ac30e78 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -65,42 +65,34 @@ class FlightMetadataReaderMock : public FlightMetadataReader { }; class FlightStreamWriterMock : public FlightStreamWriter { -public: + public: explicit FlightStreamWriterMock() = default; - Status DoneWriting() override { - return Status::OK(); - } + 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 { + Status Begin(const std::shared_ptr& schema, + const ipc::IpcWriteOptions& options) override { return Status::OK(); } - Status Begin(const std::shared_ptr &schema) override { + Status Begin(const std::shared_ptr& schema) override { return MetadataRecordBatchWriter::Begin(schema); } - ipc::WriteStats stats() const override { - return ipc::WriteStats(); - } + ipc::WriteStats stats() const override { return ipc::WriteStats(); } - Status WriteWithMetadata(const RecordBatch &batch, + Status WriteWithMetadata(const RecordBatch& batch, std::shared_ptr app_metadata) override { return Status::OK(); } - Status Close() override { - return Status::OK(); - } + Status Close() override { return Status::OK(); } - Status WriteRecordBatch(const RecordBatch &batch) override { - return Status::OK(); - } + Status WriteRecordBatch(const RecordBatch& batch) override { return Status::OK(); } }; FlightDescriptor getDescriptor(google::protobuf::Message& command) { @@ -321,46 +313,44 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { const std::string query = "query"; ON_CALL(*client_mock, DoAction) - .WillByDefault([](const FlightCallOptions& options, const Action& action, - std::unique_ptr* results) { - google::protobuf::Any command; + .WillByDefault([](const FlightCallOptions& options, const Action& action, + std::unique_ptr* results) { + google::protobuf::Any command; - pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; - prepared_statement_result.set_prepared_statement_handle("query"); + prepared_statement_result.set_prepared_statement_handle("query"); - auto schema = arrow::schema({arrow::field("id", int64())}); + auto schema = arrow::schema({arrow::field("id", int64())}); - std::shared_ptr schema_buffer; - const arrow::Result> &result = arrow::ipc::SerializeSchema( - *schema); + std::shared_ptr schema_buffer; + const arrow::Result>& result = + arrow::ipc::SerializeSchema(*schema); - ARROW_ASSIGN_OR_RAISE(schema_buffer, result); + ARROW_ASSIGN_OR_RAISE(schema_buffer, result); - prepared_statement_result.set_parameter_schema(schema_buffer->ToString()); + prepared_statement_result.set_parameter_schema(schema_buffer->ToString()); - command.PackFrom(prepared_statement_result); + command.PackFrom(prepared_statement_result); - *results = std::unique_ptr(new SimpleResultStream( - {Result{Buffer::FromString(command.SerializeAsString())}})); + *results = std::unique_ptr(new SimpleResultStream( + {Result{Buffer::FromString(command.SerializeAsString())}})); - return Status::OK(); - }); + return Status::OK(); + }); std::shared_ptr buffer_ptr; ON_CALL(*client_mock, 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(); - }); + .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(); + }); std::unique_ptr flight_info; EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); @@ -372,7 +362,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { std::shared_ptr parameter_schema; ASSERT_OK(prepared_statement->GetParameterSchema(¶meter_schema)); - arrow::Int64Builder int_builder; + arrow::Int64Builder int_builder; ASSERT_OK(int_builder.Append(1)); std::shared_ptr int_array; ASSERT_OK(int_builder.Finish(&int_array)); @@ -450,15 +440,14 @@ TEST(TestFlightSqlClient, TestGetSqlInfo) { const FlightDescriptor& descriptor = FlightDescriptor::Command(any.SerializeAsString()); FlightCallOptions call_options; - EXPECT_CALL(*client_mock, - GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void) sql_client.GetSqlInfo(call_options, sql_info, &flight_info); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + (void)sql_client.GetSqlInfo(call_options, sql_info, &flight_info); } template inline void AssertTestPreparedStatementExecuteUpdateOk(Func func) { - FlightClientMock client_mock; - std::unique_ptr client_mock_ptr(&client_mock); + auto* client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sql_client(client_mock_ptr); const std::string query = "SELECT * FROM IRRELEVANT"; @@ -473,7 +462,7 @@ inline void AssertTestPreparedStatementExecuteUpdateOk(Func func) { command.PackFrom(prepared_statement_result); const std::string& serializedCommand = command.SerializeAsString(); - ON_CALL(client_mock, DoAction) + ON_CALL(*client_mock, DoAction) .WillByDefault([&serializedCommand](const FlightCallOptions& options, const Action& action, std::unique_ptr* results) { @@ -481,9 +470,9 @@ inline void AssertTestPreparedStatementExecuteUpdateOk(Func func) { new SimpleResultStream({Result{Buffer::FromString(serializedCommand)}})); return Status::OK(); }); - EXPECT_CALL(client_mock, DoAction(_, _, _)).Times(2); + EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); - ON_CALL(client_mock, DoPut) + ON_CALL(*client_mock, DoPut) .WillByDefault([&buffer](const FlightCallOptions& options, const FlightDescriptor& descriptor1, const std::shared_ptr& schema, @@ -493,32 +482,54 @@ inline void AssertTestPreparedStatementExecuteUpdateOk(Func func) { writer->reset(new FlightStreamWriterMock()); return Status::OK(); }); - EXPECT_CALL(client_mock, DoPut(_, _, _, _, _)); int64_t rows; std::shared_ptr> prepared_statement; ASSERT_OK(sql_client.Prepare(call_options, query, &prepared_statement)); - func(prepared_statement, client_mock); + func(prepared_statement, *client_mock, query); ASSERT_OK(prepared_statement->ExecuteUpdate(&rows)); - EXPECT_EQ(expected_rows, rows); + ASSERT_EQ(expected_rows, rows); } TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { AssertTestPreparedStatementExecuteUpdateOk( [](const std::shared_ptr>& - prepared_statement, FlightClientMock& client_mock) {}); + prepared_statement, + FlightClientMock& client_mock, const std::string& query) {}); } TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding) { AssertTestPreparedStatementExecuteUpdateOk( [](const std::shared_ptr>& - prepared_statement, FlightClientMock& client_mock) { - const auto& schema = arrow::schema({arrow::field("field0", arrow::utf8()), - arrow::field("field1", arrow::uint8())}); + prepared_statement, + FlightClientMock& client_mock, const std::string& query) { + const auto schema = arrow::schema({arrow::field("field0", arrow::utf8()), + arrow::field("field1", arrow::uint8())}); + ON_CALL(client_mock, DoAction) + .WillByDefault([&schema, &query](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); + 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 stringArray; std::shared_ptr uInt8Array; - const std::vector stringData {"Loren", "Ipsum", "Foo", "Bar", "Baz"}; - const std::vector uInt8Data {0, 10, 15, 20, 25}; + const std::vector stringData{"Loren", "Ipsum", "Foo", "Bar", "Baz"}; + const std::vector uInt8Data{0, 10, 15, 20, 25}; ArrayFromVector(stringData, &stringArray); ArrayFromVector(uInt8Data, &uInt8Array); std::shared_ptr recordBatch = From b9205c1de5b11e89c3c3bee5d24305f2ba8bd4ad Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 13 Oct 2021 15:32:41 -0300 Subject: [PATCH 144/237] Refactor ExecuteUpdate with parameter binding test --- .../arrow/flight/flight-sql/client_test.cc | 45 +++++++++---------- 1 file changed, 21 insertions(+), 24 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index d741ac30e78..cfb860c7811 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -462,12 +462,28 @@ inline void AssertTestPreparedStatementExecuteUpdateOk(Func func) { command.PackFrom(prepared_statement_result); const std::string& serializedCommand = command.SerializeAsString(); + const auto schema = arrow::schema({arrow::field("field0", arrow::utf8()), + arrow::field("field1", arrow::uint8())}); + ON_CALL(*client_mock, DoAction) - .WillByDefault([&serializedCommand](const FlightCallOptions& options, + .WillByDefault([&query, &schema](const FlightCallOptions& options, const Action& action, std::unique_ptr* results) { - *results = std::unique_ptr( - new SimpleResultStream({Result{Buffer::FromString(serializedCommand)}})); + google::protobuf::Any command; + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + + prepared_statement_result.set_prepared_statement_handle(query); + 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(*client_mock, DoAction(_, _, _)).Times(2); @@ -505,27 +521,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding FlightClientMock& client_mock, const std::string& query) { const auto schema = arrow::schema({arrow::field("field0", arrow::utf8()), arrow::field("field1", arrow::uint8())}); - ON_CALL(client_mock, DoAction) - .WillByDefault([&schema, &query](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); - 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 stringArray; std::shared_ptr uInt8Array; const std::vector stringData{"Loren", "Ipsum", "Foo", "Bar", "Baz"}; @@ -535,6 +531,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding std::shared_ptr recordBatch = RecordBatch::Make(schema, 100, {stringArray, uInt8Array}); ASSERT_OK(prepared_statement->SetParameters(recordBatch)); + EXPECT_CALL(client_mock, DoPut(_, _, schema, _, _)); }); } From ade534d655d4ad99da920db891015774a034f2df Mon Sep 17 00:00:00 2001 From: Abner Eduardo Ferreira Date: Wed, 13 Oct 2021 16:04:20 -0300 Subject: [PATCH 145/237] Fix SIGSEV in test case for PreparedStatement.ExecuteUpdate with parameter binding --- .../arrow/flight/flight-sql/client_test.cc | 51 ++++++++++--------- 1 file changed, 28 insertions(+), 23 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index cfb860c7811..8f14072fd25 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -445,7 +445,8 @@ TEST(TestFlightSqlClient, TestGetSqlInfo) { } template -inline void AssertTestPreparedStatementExecuteUpdateOk(Func func) { +inline void AssertTestPreparedStatementExecuteUpdateOk( + Func func, const std::shared_ptr* schema) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); FlightSqlClientT sql_client(client_mock_ptr); @@ -462,24 +463,24 @@ inline void AssertTestPreparedStatementExecuteUpdateOk(Func func) { command.PackFrom(prepared_statement_result); const std::string& serializedCommand = command.SerializeAsString(); - const auto schema = arrow::schema({arrow::field("field0", arrow::utf8()), - arrow::field("field1", arrow::uint8())}); - ON_CALL(*client_mock, DoAction) .WillByDefault([&query, &schema](const FlightCallOptions& options, - const Action& action, - std::unique_ptr* results) { + 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); - std::shared_ptr schema_buffer; - const arrow::Result>& result = - arrow::ipc::SerializeSchema(*schema); - ARROW_ASSIGN_OR_RAISE(schema_buffer, result); + 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()); + } - 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())}})); @@ -498,42 +499,46 @@ inline void AssertTestPreparedStatementExecuteUpdateOk(Func func) { writer->reset(new FlightStreamWriterMock()); return Status::OK(); }); + if (schema == NULLPTR) { + EXPECT_CALL(*client_mock, DoPut(_, _, _, _, _)); + } else { + EXPECT_CALL(*client_mock, DoPut(_, _, *schema, _, _)); + } int64_t rows; std::shared_ptr> prepared_statement; ASSERT_OK(sql_client.Prepare(call_options, query, &prepared_statement)); - func(prepared_statement, *client_mock, query); + func(prepared_statement, *client_mock, schema); ASSERT_OK(prepared_statement->ExecuteUpdate(&rows)); ASSERT_EQ(expected_rows, rows); } -TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdate) { +TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateNoParameterBinding) { AssertTestPreparedStatementExecuteUpdateOk( [](const std::shared_ptr>& prepared_statement, - FlightClientMock& client_mock, const std::string& query) {}); + FlightClientMock& client_mock, const std::shared_ptr* schema) {}, + NULLPTR); } TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding) { + const auto schema = arrow::schema( + {arrow::field("field0", arrow::utf8()), arrow::field("field1", arrow::uint8())}); AssertTestPreparedStatementExecuteUpdateOk( [](const std::shared_ptr>& prepared_statement, - FlightClientMock& client_mock, const std::string& query) { - const auto schema = arrow::schema({arrow::field("field0", arrow::utf8()), - arrow::field("field1", arrow::uint8())}); - + FlightClientMock& client_mock, const std::shared_ptr* schema) { std::shared_ptr stringArray; std::shared_ptr uInt8Array; - const std::vector stringData{"Loren", "Ipsum", "Foo", "Bar", "Baz"}; + const std::vector stringData{"Lorem", "Ipsum", "Foo", "Bar", "Baz"}; const std::vector uInt8Data{0, 10, 15, 20, 25}; ArrayFromVector(stringData, &stringArray); ArrayFromVector(uInt8Data, &uInt8Array); std::shared_ptr recordBatch = - RecordBatch::Make(schema, 100, {stringArray, uInt8Array}); + RecordBatch::Make(*schema, 100, {stringArray, uInt8Array}); ASSERT_OK(prepared_statement->SetParameters(recordBatch)); - - EXPECT_CALL(client_mock, DoPut(_, _, schema, _, _)); - }); + }, + &schema); } } // namespace sql From 5d03029a092075b7ff816cf32f995f864aaf4bb0 Mon Sep 17 00:00:00 2001 From: Abner Eduardo Ferreira Date: Wed, 13 Oct 2021 17:57:26 -0300 Subject: [PATCH 146/237] Minor refactor: move variables closer to used --- .../arrow/flight/flight-sql/client_test.cc | 29 +++++++++---------- 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 8f14072fd25..88818b3b91d 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -456,12 +456,6 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( int64_t expected_rows = 100L; pb::sql::DoPutUpdateResult result; result.set_record_count(expected_rows); - auto buffer = Buffer::FromString(result.SerializeAsString()); - google::protobuf::Any command; - pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; - prepared_statement_result.set_prepared_statement_handle(query); - command.PackFrom(prepared_statement_result); - const std::string& serializedCommand = command.SerializeAsString(); ON_CALL(*client_mock, DoAction) .WillByDefault([&query, &schema](const FlightCallOptions& options, @@ -489,6 +483,7 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( }); EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); + auto buffer = Buffer::FromString(result.SerializeAsString()); ON_CALL(*client_mock, DoPut) .WillByDefault([&buffer](const FlightCallOptions& options, const FlightDescriptor& descriptor1, @@ -508,7 +503,7 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( int64_t rows; std::shared_ptr> prepared_statement; ASSERT_OK(sql_client.Prepare(call_options, query, &prepared_statement)); - func(prepared_statement, *client_mock, schema); + func(prepared_statement, *client_mock, schema, expected_rows); ASSERT_OK(prepared_statement->ExecuteUpdate(&rows)); ASSERT_EQ(expected_rows, rows); } @@ -517,7 +512,8 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateNoParameterBinding) AssertTestPreparedStatementExecuteUpdateOk( [](const std::shared_ptr>& prepared_statement, - FlightClientMock& client_mock, const std::shared_ptr* schema) {}, + FlightClientMock& client_mock, const std::shared_ptr* schema, + const int64_t& row_count) {}, NULLPTR); } @@ -527,15 +523,16 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding AssertTestPreparedStatementExecuteUpdateOk( [](const std::shared_ptr>& prepared_statement, - FlightClientMock& client_mock, const std::shared_ptr* schema) { - std::shared_ptr stringArray; - std::shared_ptr uInt8Array; - const std::vector stringData{"Lorem", "Ipsum", "Foo", "Bar", "Baz"}; - const std::vector uInt8Data{0, 10, 15, 20, 25}; - ArrayFromVector(stringData, &stringArray); - ArrayFromVector(uInt8Data, &uInt8Array); + FlightClientMock& client_mock, const std::shared_ptr* schema, + const int64_t& row_count) { + std::shared_ptr string_array; + std::shared_ptr uint8_array; + const std::vector string_data{"Lorem", "Ipsum", "Foo", "Bar", "Baz"}; + const std::vector uint8_data{0, 10, 15, 20, 25}; + ArrayFromVector(string_data, &string_array); + ArrayFromVector(uint8_data, &uint8_array); std::shared_ptr recordBatch = - RecordBatch::Make(*schema, 100, {stringArray, uInt8Array}); + RecordBatch::Make(*schema, row_count, {string_array, uint8_array}); ASSERT_OK(prepared_statement->SetParameters(recordBatch)); }, &schema); From b30041539f491f31df46abd94e121cd870931905 Mon Sep 17 00:00:00 2001 From: Abner Eduardo Ferreira Date: Thu, 14 Oct 2021 14:15:38 -0300 Subject: [PATCH 147/237] Fix resource leak where record batch is being created for client_impl.h --- cpp/src/arrow/flight/flight-sql/client_impl.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index eb77a4139f5..5c4158fddb3 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -332,8 +332,9 @@ Status PreparedStatementT::ExecuteUpdate(int64_t* rows) { } else { const std::shared_ptr schema = arrow::schema({}); ARROW_RETURN_NOT_OK(client->DoPut(options, descriptor, schema, &writer, &reader)); - ARROW_RETURN_NOT_OK(writer->WriteRecordBatch( - *arrow::RecordBatch::Make(schema, 0, (std::vector>){}))); + 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()); From a89a8ffec3a0080cc0ab28fe78599c50afc6c782 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 14 Oct 2021 17:28:38 -0300 Subject: [PATCH 148/237] Fix CheckStyle issues --- cpp/src/arrow/flight/flight-sql/client.h | 3 +- cpp/src/arrow/flight/flight-sql/client_impl.h | 25 +++---- .../arrow/flight/flight-sql/client_test.cc | 2 +- .../flight-sql/example/sqlite_server.cc | 34 ++++----- .../flight/flight-sql/example/sqlite_server.h | 70 +++++++++---------- .../flight-sql/example/sqlite_statement.cc | 6 +- .../example/sqlite_statement_batch_reader.cc | 11 ++- .../example/sqlite_statement_batch_reader.h | 7 +- .../sqlite_tables_schema_batch_reader.cpp | 4 +- .../arrow/flight/flight-sql/sql_server.cpp | 42 +++++------ .../flight/flight-sql/sql_server_test.cc | 26 ++++--- cpp/src/arrow/flight/flight-sql/test_app.cc | 7 +- 12 files changed, 118 insertions(+), 119 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 4690958c8f1..e7fa511ea2a 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -204,8 +204,7 @@ class FlightSqlClientT { /// \param[in] sql_info the SQL info required. /// \param[out] flight_info The FlightInfo describing where to access the dataset. /// \return Status. - Status GetSqlInfo(const FlightCallOptions& options, - const std::vector& sql_info, + Status GetSqlInfo(const FlightCallOptions& options, const std::vector& sql_info, std::unique_ptr* flight_info) const; /// \brief Request a list of SQL information. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 5c4158fddb3..f7983110434 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -332,8 +332,8 @@ Status PreparedStatementT::ExecuteUpdate(int64_t* rows) { } 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>){}); + const auto& record_batch = + arrow::RecordBatch::Make(schema, 0, (std::vector>){}); ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*record_batch)); } @@ -363,8 +363,8 @@ bool PreparedStatementT::IsClosed() const { } template -Status PreparedStatementT::GetResultSetSchema(std::shared_ptr *schema) { - auto &args = prepared_statement_result.dataset_schema(); +Status PreparedStatementT::GetResultSetSchema(std::shared_ptr* schema) { + auto& args = prepared_statement_result.dataset_schema(); std::shared_ptr schema_buffer = std::make_shared(args); io::BufferReader reader(schema_buffer); @@ -377,7 +377,7 @@ Status PreparedStatementT::GetResultSetSchema(std::shared_ptr *schema template Status PreparedStatementT::GetParameterSchema(std::shared_ptr* schema) { - auto &args = prepared_statement_result.parameter_schema(); + auto& args = prepared_statement_result.parameter_schema(); std::shared_ptr schema_buffer = std::make_shared(args); io::BufferReader reader(schema_buffer); @@ -414,19 +414,20 @@ Status PreparedStatementT::Close() { } template -Status FlightSqlClientT::GetSqlInfo( - const FlightCallOptions& options, const std::vector& sql_info, - std::unique_ptr* flight_info) const { +Status FlightSqlClientT::GetSqlInfo(const FlightCallOptions& options, + const std::vector& sql_info, + std::unique_ptr* flight_info) const { pb::sql::CommandGetSqlInfo command; for (const int& info : sql_info) command.add_info(info); return GetFlightInfoForCommand(client, options, flight_info, command); } template -Status FlightSqlClientT::GetSqlInfo( - const FlightCallOptions& options, const std::vector& sql_info, - std::unique_ptr* flight_info) const { - return GetSqlInfo(options, reinterpret_cast&>(sql_info), flight_info); +Status FlightSqlClientT::GetSqlInfo(const FlightCallOptions& options, + const std::vector& sql_info, + std::unique_ptr* flight_info) const { + return GetSqlInfo(options, reinterpret_cast&>(sql_info), + flight_info); } } // namespace internal diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 88818b3b91d..457aa37718e 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -66,7 +66,7 @@ class FlightMetadataReaderMock : public FlightMetadataReader { class FlightStreamWriterMock : public FlightStreamWriter { public: - explicit FlightStreamWriterMock() = default; + FlightStreamWriterMock() = default; Status DoneWriting() override { return Status::OK(); } diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index e75c4f3e557..ce2e6cc9e3c 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -15,19 +15,21 @@ // specific language governing permissions and limitations // under the License. +#include "arrow/flight/flight-sql/example/sqlite_server.h" + +#include + #include #include #include #include -#include #include #include "arrow/api.h" -#include "arrow/flight/flight-sql/sql_server.h" -#include "arrow/flight/flight-sql/example/sqlite_server.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" #include "arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h" +#include "arrow/flight/flight-sql/sql_server.h" namespace arrow { namespace flight { @@ -138,8 +140,7 @@ Status DoGetSQLiteQuery(sqlite3* db, const std::string& query, ARROW_RETURN_NOT_OK(SqliteStatement::Create(db, query, &statement)); std::shared_ptr reader; - ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create( - statement, schema, &reader)); + ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create(statement, schema, &reader)); *result = std::unique_ptr(new RecordBatchStream(reader)); @@ -295,11 +296,10 @@ Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& comma if (command.include_schema()) { std::shared_ptr table_schema_reader = std::make_shared(reader, query, db_); - *result = std::unique_ptr( - new RecordBatchStream(table_schema_reader)); + *result = + std::unique_ptr(new RecordBatchStream(table_schema_reader)); } else { - *result = std::unique_ptr( - new RecordBatchStream(reader)); + *result = std::unique_ptr(new RecordBatchStream(reader)); } return Status::OK(); @@ -484,7 +484,8 @@ Status SQLiteFlightSqlServer::DoPutPreparedStatement( break; } case Type::STRING: { - std::shared_ptr buffer = reinterpret_cast(*holder).value; + std::shared_ptr buffer = + reinterpret_cast(*holder).value; const std::string string = buffer->ToString(); const char* value = string.c_str(); sqlite3_bind_text(stmt, c + 1, value, static_cast(strlen(value)), @@ -492,7 +493,8 @@ Status SQLiteFlightSqlServer::DoPutPreparedStatement( break; } case Type::BINARY: { - std::shared_ptr buffer = reinterpret_cast(*holder).value; + std::shared_ptr buffer = + reinterpret_cast(*holder).value; sqlite3_bind_blob(stmt, c + 1, buffer->data(), static_cast(buffer->size()), SQLITE_TRANSIENT); break; @@ -530,10 +532,9 @@ Status SQLiteFlightSqlServer::GetFlightInfoPrimaryKeys( SqlSchema::GetPrimaryKeysSchema()); } -Status -SQLiteFlightSqlServer::DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys &command, - const ServerCallContext &context, - std::unique_ptr *result) { +Status SQLiteFlightSqlServer::DoGetPrimaryKeys( + const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, + std::unique_ptr* result) { std::stringstream table_query; // The field key_name can not be recovered by the sqlite, so it is being set @@ -587,7 +588,8 @@ std::string PrepareQueryForGetImportedOrExportedKeys(const std::string& filter) 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 + WHERE m.type = 'table') WHERE )" + + filter + R"( ORDER BY pk_catalog_name, pk_schema_name, pk_table_name, pk_key_name, key_sequence)"; } diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index db446141a39..cf63e939eac 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -41,10 +41,10 @@ std::shared_ptr GetArrowType(const char* sqlite_type); /// \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()), + field("string", utf8()), + field("bytes", binary()), + field("bigint", int64()), + field("double", float64()), }); } @@ -98,7 +98,7 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { const ServerCallContext& context, std::unique_ptr* result) override; Status DoPutPreparedStatement(const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext &context, + const ServerCallContext& context, std::unique_ptr& reader, std::unique_ptr& writer) override; @@ -110,36 +110,36 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { Status DoGetTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, std::unique_ptr* result) override; - Status GetFlightInfoTableTypes(const ServerCallContext &context, - const FlightDescriptor &descriptor, - std::unique_ptr *info) override; - Status DoGetTableTypes(const ServerCallContext &context, - std::unique_ptr *result) override; - Status GetFlightInfoImportedKeys(const pb::sql::CommandGetImportedKeys &command, - const ServerCallContext &context, - const FlightDescriptor &descriptor, - std::unique_ptr *info) override; - Status DoGetImportedKeys(const pb::sql::CommandGetImportedKeys &command, - const ServerCallContext &context, - std::unique_ptr *result) override; - Status GetFlightInfoExportedKeys(const pb::sql::CommandGetExportedKeys &command, - const ServerCallContext &context, - const FlightDescriptor &descriptor, - std::unique_ptr *info) override; - Status DoGetExportedKeys(const pb::sql::CommandGetExportedKeys &command, - const ServerCallContext &context, - std::unique_ptr *result) override; - - Status GetFlightInfoPrimaryKeys(const pb::sql::CommandGetPrimaryKeys &command, - const ServerCallContext &context, - const FlightDescriptor &descriptor, - std::unique_ptr *info) override; - - Status DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys &command, - const ServerCallContext &context, - std::unique_ptr *result) override; - -private: + Status GetFlightInfoTableTypes(const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) override; + Status DoGetTableTypes(const ServerCallContext& context, + std::unique_ptr* result) override; + Status GetFlightInfoImportedKeys(const pb::sql::CommandGetImportedKeys& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) override; + Status DoGetImportedKeys(const pb::sql::CommandGetImportedKeys& command, + const ServerCallContext& context, + std::unique_ptr* result) override; + Status GetFlightInfoExportedKeys(const pb::sql::CommandGetExportedKeys& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) override; + Status DoGetExportedKeys(const pb::sql::CommandGetExportedKeys& command, + const ServerCallContext& context, + std::unique_ptr* result) override; + + Status GetFlightInfoPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) override; + + Status DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, + const ServerCallContext& context, + std::unique_ptr* result) override; + + private: sqlite3* db_; boost::uuids::random_generator uuid_generator_; std::map> prepared_statements_; diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc index da1626516f7..d983ada5aee 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc @@ -66,9 +66,9 @@ Status SqliteStatement::GetSchema(std::shared_ptr* schema) const { 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: + // 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 diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index b29be607efe..7d30a2ec145 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -72,9 +72,8 @@ namespace example { 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)) { + std::shared_ptr statement, std::shared_ptr schema) + : statement_(std::move(statement)), schema_(std::move(schema)) { rc = SQLITE_OK; } @@ -93,9 +92,9 @@ Status SqliteStatementBatchReader::Create( } Status SqliteStatementBatchReader::Create( - const std::shared_ptr &statement_, - const std::shared_ptr &schema, - std::shared_ptr *result) { + const std::shared_ptr& statement_, + const std::shared_ptr& schema, + std::shared_ptr* result) { result->reset(new SqliteStatementBatchReader(statement_, schema)); return Status::OK(); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h index a92281ead06..d8f25025687 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h @@ -18,7 +18,6 @@ #pragma once #include -#include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/api.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" @@ -42,9 +41,9 @@ class SqliteStatementBatchReader : public RecordBatchReader { /// \param[in] schema Schema to be used on results. /// \param[out] result The resulting RecordBatchReader. /// \return Status. - static Status Create(const std::shared_ptr &statement, - const std::shared_ptr &schema, - std::shared_ptr *result); + static Status Create(const std::shared_ptr& statement, + const std::shared_ptr& schema, + std::shared_ptr* result); std::shared_ptr schema() const override; diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp index 4a20b5de308..62bb98667d5 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp @@ -78,9 +78,7 @@ Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* int nullable = sqlite3_column_int(schema_statement->GetSqlite3Stmt(), 3); column_fields.push_back( - arrow::field(column_name, - GetArrowType(column_type), - nullable == 0, NULL)); + arrow::field(column_name, GetArrowType(column_type), nullable == 0, NULL)); } } const arrow::Result>& value = diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/sql_server.cpp index 57c4963a71e..98de2f794c4 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/sql_server.cpp @@ -128,11 +128,10 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket return Status::Invalid("The defined request is invalid."); } -Status FlightSqlServerBase::DoPut(const ServerCallContext &context, +Status FlightSqlServerBase::DoPut(const ServerCallContext& context, std::unique_ptr reader, std::unique_ptr writer) { - - const FlightDescriptor &request = reader->descriptor(); + const FlightDescriptor& request = reader->descriptor(); google::protobuf::Any any; any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size())); @@ -313,10 +312,10 @@ Status FlightSqlServerBase::ClosePreparedStatement( return Status::NotImplemented("ClosePreparedStatement not implemented"); } -Status FlightSqlServerBase::DoPutPreparedStatement(const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext &context, - std::unique_ptr &reader, - std::unique_ptr &writer) { +Status FlightSqlServerBase::DoPutPreparedStatement( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, std::unique_ptr& reader, + std::unique_ptr& writer) { return Status::NotImplemented("DoPutPreparedStatement not implemented"); } @@ -352,27 +351,20 @@ std::shared_ptr SqlSchema::GetTableTypesSchema() { } std::shared_ptr SqlSchema::GetPrimaryKeysSchema() { -return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8()), - field("table_name", utf8()), field("column_name", utf8()), - field("key_sequence", int64()), field("key_name", utf8())}); + return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8()), + field("table_name", utf8()), field("column_name", utf8()), + field("key_sequence", int64()), field("key_name", utf8())}); } std::shared_ptr SqlSchema::GetImportedAndExportedKeysSchema() { - return arrow::schema({ - field("pk_catalog_name", utf8(), true), - field("pk_schema_name", utf8(), true), - field("pk_table_name", utf8(), false), - field("pk_column_name", utf8(), false), - field("fk_catalog_name", utf8(), true), - field("fk_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) - }); + return arrow::schema( + {field("pk_catalog_name", utf8(), true), field("pk_schema_name", utf8(), true), + field("pk_table_name", utf8(), false), field("pk_column_name", utf8(), false), + field("fk_catalog_name", utf8(), true), field("fk_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)}); } } // namespace sql diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc index fc3729ae5d8..e0b6d6c7606 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/sql_server_test.cc @@ -18,8 +18,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -139,7 +139,12 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { DECLARE_NULL_ARRAY(catalog_name, String, 3); DECLARE_NULL_ARRAY(schema_name, String, 3); - DECLARE_ARRAY(table_name, String, ({"foreignTable", "intTable", "sqlite_sequence", })); + DECLARE_ARRAY(table_name, String, + ({ + "foreignTable", + "intTable", + "sqlite_sequence", + })); DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); const std::shared_ptr
& expected_table = Table::Make( @@ -206,7 +211,12 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { DECLARE_NULL_ARRAY(catalog_name, String, 3); DECLARE_NULL_ARRAY(schema_name, String, 3); - DECLARE_ARRAY(table_name, String, ({"foreignTable", "intTable", "sqlite_sequence", })); + DECLARE_ARRAY(table_name, String, + ({ + "foreignTable", + "intTable", + "sqlite_sequence", + })); DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); const std::shared_ptr
& expected_table = Table::Make( @@ -360,8 +370,8 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) std::shared_ptr parameter_schema; ASSERT_OK(prepared_statement->GetParameterSchema(¶meter_schema)); - const std::shared_ptr& expected_parameter_schema = arrow::schema( - {arrow::field("parameter_1", example::GetUnknownColumnDataType())}); + const std::shared_ptr& expected_parameter_schema = + arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); ASSERT_TRUE(expected_parameter_schema->Equals(*parameter_schema)); @@ -417,8 +427,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { std::unique_ptr flight_info; std::vector table_types; - ASSERT_OK(sql_client->GetPrimaryKeys({}, nullptr, nullptr, "int%", - &flight_info)); + ASSERT_OK(sql_client->GetPrimaryKeys({}, nullptr, nullptr, "int%", &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); @@ -505,8 +514,7 @@ TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { ASSERT_TRUE(expected_table->Equals(*table)); } -auto env = - ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); +auto env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app.cc index a8bc19fa385..a83e696c627 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app.cc @@ -134,7 +134,8 @@ Status RunMain() { } else if (fLS::FLAGS_command == "PreparedStatementExecute") { std::shared_ptr prepared_statement; - ARROW_RETURN_NOT_OK(sqlClient.Prepare(call_options, fLS::FLAGS_query, &prepared_statement)); + ARROW_RETURN_NOT_OK( + sqlClient.Prepare(call_options, fLS::FLAGS_query, &prepared_statement)); ARROW_RETURN_NOT_OK(prepared_statement->Execute(&info)); ARROW_RETURN_NOT_OK(PrintResults(sqlClient, call_options, info)); } else if (fLS::FLAGS_command == "PreparedStatementExecuteParameterBinding") { @@ -146,7 +147,7 @@ Status RunMain() { ARROW_RETURN_NOT_OK(prepared_statement->GetResultSetSchema(&result_set_schema)); std::cout << result_set_schema->ToString(false) << std::endl; - arrow::Int64Builder int_builder; + 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)); @@ -156,7 +157,7 @@ Status RunMain() { ARROW_RETURN_NOT_OK(prepared_statement->SetParameters(result)); ARROW_RETURN_NOT_OK(prepared_statement->Execute(&info)); ARROW_RETURN_NOT_OK(PrintResults(sqlClient, call_options, info)); - }else if (fLS::FLAGS_command == "GetSchemas") { + } else if (fLS::FLAGS_command == "GetSchemas") { ARROW_RETURN_NOT_OK(sqlClient.GetSchemas(call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, &info)); } else if (fLS::FLAGS_command == "GetTableTypes") { From 6248009942d05bd955c1f6e4ef4a67c8ac3311bb Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 14 Oct 2021 17:59:19 -0300 Subject: [PATCH 149/237] Rename server files --- cpp/src/arrow/flight/flight-sql/CMakeLists.txt | 10 +++++----- .../arrow/flight/flight-sql/example/sqlite_server.cc | 2 +- .../arrow/flight/flight-sql/example/sqlite_server.h | 2 +- .../example/sqlite_tables_schema_batch_reader.cpp | 2 +- .../flight/flight-sql/{sql_server.cpp => server.cpp} | 2 +- .../arrow/flight/flight-sql/{sql_server.h => server.h} | 3 +-- .../flight-sql/{sql_server_test.cc => server_test.cc} | 2 +- .../flight/flight-sql/{test_app.cc => test_app_cli.cc} | 0 .../flight-sql/{test_server.cc => test_server_cli.cc} | 0 9 files changed, 11 insertions(+), 12 deletions(-) rename cpp/src/arrow/flight/flight-sql/{sql_server.cpp => server.cpp} (99%) rename cpp/src/arrow/flight/flight-sql/{sql_server.h => server.h} (99%) rename cpp/src/arrow/flight/flight-sql/{sql_server_test.cc => server_test.cc} (99%) rename cpp/src/arrow/flight/flight-sql/{test_app.cc => test_app_cli.cc} (100%) rename cpp/src/arrow/flight/flight-sql/{test_server.cc => test_server_cli.cc} (100%) diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 7334445dab2..965aeae9305 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -46,7 +46,7 @@ set(CMAKE_CXX_FLAGS_BACKUP "${CMAKE_CXX_FLAGS}") string(REPLACE "/WX" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") string(REPLACE "-Werror " " " CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") -set(ARROW_FLIGHT_SQL_SRCS sql_server.cpp) +set(ARROW_FLIGHT_SQL_SRCS server.cpp) add_arrow_lib(arrow_flight_sql CMAKE_PACKAGE_NAME @@ -82,7 +82,7 @@ endif() add_arrow_test(flight_sql_test SOURCES client_test.cc - sql_server_test.cc + server_test.cc "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" STATIC_LINK_LIBS ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} @@ -91,7 +91,7 @@ add_arrow_test(flight_sql_test LABELS "arrow_flight_sql") -add_executable(flight_sql_test_app test_app.cc +add_executable(flight_sql_test_app test_app_cli.cc ${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc) target_link_libraries(flight_sql_test_app PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES}) @@ -100,8 +100,8 @@ find_package(SQLite3) include_directories(${SQLite3_INCLUDE_DIRS}) add_executable(flight_sql_test_server - sql_server.cpp - test_server.cc + server.cpp + test_server_cli.cc example/sqlite_statement.cc example/sqlite_statement_batch_reader.cc example/sqlite_server.cc diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index ce2e6cc9e3c..689676c0a49 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -29,7 +29,7 @@ #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" #include "arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h" -#include "arrow/flight/flight-sql/sql_server.h" +#include "arrow/flight/flight-sql/server.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index cf63e939eac..f5fcdb04143 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -25,7 +25,7 @@ #include "arrow/api.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" #include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" -#include "arrow/flight/flight-sql/sql_server.h" +#include "arrow/flight/flight-sql/server.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp index 62bb98667d5..ddb4e2e6c96 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp @@ -17,7 +17,7 @@ #include "arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h" -#include +#include #include #include #include diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.cpp b/cpp/src/arrow/flight/flight-sql/server.cpp similarity index 99% rename from cpp/src/arrow/flight/flight-sql/sql_server.cpp rename to cpp/src/arrow/flight/flight-sql/server.cpp index 98de2f794c4..70c3069b376 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.cpp +++ b/cpp/src/arrow/flight/flight-sql/server.cpp @@ -18,7 +18,7 @@ // Interfaces to use for defining Flight RPC servers. API should be considered // experimental for now -#include "arrow/flight/flight-sql/sql_server.h" +#include "arrow/flight/flight-sql/server.h" #include "arrow/buffer.h" diff --git a/cpp/src/arrow/flight/flight-sql/sql_server.h b/cpp/src/arrow/flight/flight-sql/server.h similarity index 99% rename from cpp/src/arrow/flight/flight-sql/sql_server.h rename to cpp/src/arrow/flight/flight-sql/server.h index e1adb09b438..2a1c56f3538 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server.h +++ b/cpp/src/arrow/flight/flight-sql/server.h @@ -21,10 +21,9 @@ #pragma once #include +#include #include -#include "../server.h" - namespace pb = arrow::flight::protocol; namespace arrow { diff --git a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc b/cpp/src/arrow/flight/flight-sql/server_test.cc similarity index 99% rename from cpp/src/arrow/flight/flight-sql/sql_server_test.cc rename to cpp/src/arrow/flight/flight-sql/server_test.cc index e0b6d6c7606..482cd538a9e 100644 --- a/cpp/src/arrow/flight/flight-sql/sql_server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/server_test.cc @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/cpp/src/arrow/flight/flight-sql/test_app.cc b/cpp/src/arrow/flight/flight-sql/test_app_cli.cc similarity index 100% rename from cpp/src/arrow/flight/flight-sql/test_app.cc rename to cpp/src/arrow/flight/flight-sql/test_app_cli.cc diff --git a/cpp/src/arrow/flight/flight-sql/test_server.cc b/cpp/src/arrow/flight/flight-sql/test_server_cli.cc similarity index 100% rename from cpp/src/arrow/flight/flight-sql/test_server.cc rename to cpp/src/arrow/flight/flight-sql/test_server_cli.cc From d0e94764bbfc0dc0f6485b122b875f6de1d6e58c Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 18 Oct 2021 14:21:42 -0300 Subject: [PATCH 150/237] Fix wrong error messages on server.cpp --- cpp/src/arrow/flight/flight-sql/server.cpp | 52 +++++++++++----------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/server.cpp b/cpp/src/arrow/flight/flight-sql/server.cpp index 70c3069b376..e83aa61e198 100644 --- a/cpp/src/arrow/flight/flight-sql/server.cpp +++ b/cpp/src/arrow/flight/flight-sql/server.cpp @@ -183,86 +183,80 @@ Status FlightSqlServerBase::DoAction(const ServerCallContext& context, Status FlightSqlServerBase::GetFlightInfoCatalogs(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return Status::NotImplemented("GetFlightInfoCatalogs not Implemented"); + return Status::NotImplemented("GetFlightInfoCatalogs not implemented"); } -Status FlightSqlServerBase::DoGetImportedKeys( - const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, - std::unique_ptr* result) { - return Status::NotImplemented("DoGetImportedKeys not implemented"); +Status FlightSqlServerBase::DoGetCatalogs(const ServerCallContext& context, + std::unique_ptr* result) { + return Status::NotImplemented("DoGetCatalogs not implemented"); } Status FlightSqlServerBase::GetFlightInfoStatement( const pb::sql::CommandStatementQuery& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return Status::NotImplemented("DoGetImportedKeys not implemented"); + return Status::NotImplemented("GetFlightInfoStatement not implemented"); } Status FlightSqlServerBase::DoGetStatement(const pb::sql::TicketStatementQuery& command, const ServerCallContext& context, std::unique_ptr* result) { - return Status::NotImplemented("GetFlightInfoStatement not implemented"); + return Status::NotImplemented("DoGetStatement not implemented"); } Status FlightSqlServerBase::GetFlightInfoPreparedStatement( const pb::sql::CommandPreparedStatementQuery& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return Status::NotImplemented("DoGetStatement not implemented"); + return Status::NotImplemented("GetFlightInfoPreparedStatement not implemented"); } Status FlightSqlServerBase::DoGetPreparedStatement( const pb::sql::CommandPreparedStatementQuery& command, const ServerCallContext& context, std::unique_ptr* result) { - return Status::NotImplemented("GetFlightInfoPreparedStatement not implemented"); -} - -Status FlightSqlServerBase::DoGetCatalogs(const ServerCallContext& context, - std::unique_ptr* result) { - return Status::NotImplemented("DoGetCatalogs not implemented"); + return Status::NotImplemented("DoGetPreparedStatement not implemented"); } Status FlightSqlServerBase::GetFlightInfoSqlInfo( const pb::sql::CommandGetSqlInfo& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return Status::NotImplemented("DoGetPreparedStatement not implemented"); + return Status::NotImplemented("GetFlightInfoSqlInfo not implemented"); } Status FlightSqlServerBase::DoGetSqlInfo(const pb::sql::CommandGetSqlInfo& command, const ServerCallContext& context, std::unique_ptr* result) { - return Status::NotImplemented("GetFlightInfoSqlInfo not implemented"); + return Status::NotImplemented("DoGetSqlInfo not implemented"); } Status FlightSqlServerBase::GetFlightInfoSchemas( const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return Status::NotImplemented("DoGetSqlInfo not implemented"); + return Status::NotImplemented("GetFlightInfoSchemas not implemented"); } Status FlightSqlServerBase::DoGetSchemas(const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, std::unique_ptr* result) { - return Status::NotImplemented("GetFlightInfoSchemas not implemented"); + return Status::NotImplemented("DoGetSchemas not implemented"); } Status FlightSqlServerBase::GetFlightInfoTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return Status::NotImplemented("DoGetSchemas not implemented"); + return Status::NotImplemented("GetFlightInfoTables not implemented"); } Status FlightSqlServerBase::DoGetTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, std::unique_ptr* result) { - return Status::NotImplemented("GetFlightInfoTables not implemented"); + return Status::NotImplemented("DoGetTables not implemented"); } Status FlightSqlServerBase::GetFlightInfoTableTypes(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return Status::NotImplemented("DoGetTables not implemented"); + return Status::NotImplemented("GetFlightInfoTableTypes not implemented"); } Status FlightSqlServerBase::DoGetTableTypes(const ServerCallContext& context, @@ -273,31 +267,37 @@ Status FlightSqlServerBase::DoGetTableTypes(const ServerCallContext& context, Status FlightSqlServerBase::GetFlightInfoPrimaryKeys( const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return Status::NotImplemented("GetFlightInfoTableTypes not implemented"); + return Status::NotImplemented("GetFlightInfoPrimaryKeys not implemented"); } Status FlightSqlServerBase::DoGetPrimaryKeys( const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, std::unique_ptr* result) { - return Status::NotImplemented("GetFlightInfoPrimaryKeys not implemented"); + return Status::NotImplemented("DoGetPrimaryKeys not implemented"); } Status FlightSqlServerBase::GetFlightInfoExportedKeys( const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return Status::NotImplemented("DoGetPrimaryKeys not implemented"); + return Status::NotImplemented("GetFlightInfoExportedKeys not implemented"); } Status FlightSqlServerBase::DoGetExportedKeys( const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, std::unique_ptr* result) { - return Status::NotImplemented("GetFlightInfoExportedKeys not implemented"); + return Status::NotImplemented("DoGetExportedKeys not implemented"); } Status FlightSqlServerBase::GetFlightInfoImportedKeys( const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return Status::NotImplemented("DoGetExportedKeys not implemented"); + return Status::NotImplemented("GetFlightInfoImportedKeys not implemented"); +} + +Status FlightSqlServerBase::DoGetImportedKeys( + const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, + std::unique_ptr* result) { + return Status::NotImplemented("DoGetImportedKeys not implemented"); } Status FlightSqlServerBase::CreatePreparedStatement( From 84ae269e1f131e175bc254cb7bef408b957870a7 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 18 Oct 2021 16:10:10 -0300 Subject: [PATCH 151/237] Implement CommandPreparedStatementUpdate on SQL server example (#169) * Implement CommandPreparedStatementUpdate on SQL server example * Refactor sqlite_server to avoid duplication --- .../flight-sql/example/sqlite_server.cc | 142 +++++++++++------- .../flight/flight-sql/example/sqlite_server.h | 14 +- cpp/src/arrow/flight/flight-sql/server.cpp | 17 ++- cpp/src/arrow/flight/flight-sql/server.h | 14 +- .../arrow/flight/flight-sql/server_test.cc | 99 ++++++++++++ 5 files changed, 225 insertions(+), 61 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 689676c0a49..cc6933fe8e1 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -94,6 +94,62 @@ std::string PrepareQueryForGetTables(const pb::sql::CommandGetTables& command) { return table_query.str(); } +Status SetParametersOnSQLiteStatement(sqlite3_stmt* stmt, + std::unique_ptr& 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 = reinterpret_cast(*scalar).value; + + switch (holder->type->id()) { + case Type::INT64: { + int64_t value = reinterpret_cast(*holder).value; + sqlite3_bind_int64(stmt, c + 1, value); + break; + } + case Type::FLOAT: { + double value = reinterpret_cast(*holder).value; + sqlite3_bind_double(stmt, c + 1, value); + break; + } + case Type::STRING: { + std::shared_ptr buffer = + reinterpret_cast(*holder).value; + const std::string string = buffer->ToString(); + const char* value = string.c_str(); + sqlite3_bind_text(stmt, c + 1, value, static_cast(strlen(value)), + SQLITE_TRANSIENT); + break; + } + case Type::BINARY: { + std::shared_ptr buffer = + reinterpret_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(); +} + SQLiteFlightSqlServer::SQLiteFlightSqlServer() { db_ = NULLPTR; if (sqlite3_open(":memory:", &db_)) { @@ -439,11 +495,8 @@ Status SQLiteFlightSqlServer::DoGetPreparedStatement( return Status::OK(); } -Status SQLiteFlightSqlServer::DoPutPreparedStatement( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, std::unique_ptr& reader, - std::unique_ptr& writer) { - const std::string& prepared_statement_handle = command.prepared_statement_handle(); +Status SQLiteFlightSqlServer::GetStatementByHandle( + const std::string& prepared_statement_handle, std::shared_ptr* result) { const auto& uuid = boost::lexical_cast(prepared_statement_handle); auto search = prepared_statements_.find(uuid); @@ -451,61 +504,44 @@ Status SQLiteFlightSqlServer::DoPutPreparedStatement( return Status::Invalid("Prepared statement not found"); } - std::shared_ptr statement = search->second; + *result = search->second; + return Status::OK(); +} + +Status SQLiteFlightSqlServer::DoPutPreparedStatementQuery( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, std::unique_ptr& reader, + std::unique_ptr& writer) { + const std::string& prepared_statement_handle = command.prepared_statement_handle(); + std::shared_ptr statement; + ARROW_RETURN_NOT_OK(GetStatementByHandle(prepared_statement_handle, &statement)); sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); + ARROW_RETURN_NOT_OK(SetParametersOnSQLiteStatement(stmt, reader)); - // Loading parameters received from the RecordBatches on the underlying sqlite3_stmt. - FlightStreamChunk chunk; - while (true) { - RETURN_NOT_OK(reader->Next(&chunk)); - std::shared_ptr& record_batch = chunk.data; - if (record_batch == nullptr) break; + return Status::OK(); +} - const int64_t num_rows = record_batch->num_rows(); - const int& num_columns = record_batch->num_columns(); +Status SQLiteFlightSqlServer::DoPutPreparedStatementUpdate( + const pb::sql::CommandPreparedStatementUpdate& command, + const ServerCallContext& context, std::unique_ptr& reader, + std::unique_ptr& writer) { + const std::string& prepared_statement_handle = command.prepared_statement_handle(); - 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)); + std::shared_ptr statement; + ARROW_RETURN_NOT_OK(GetStatementByHandle(prepared_statement_handle, &statement)); - auto& holder = reinterpret_cast(*scalar).value; + sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); + ARROW_RETURN_NOT_OK(SetParametersOnSQLiteStatement(stmt, reader)); - switch (holder->type->id()) { - case Type::INT64: { - int64_t value = reinterpret_cast(*holder).value; - sqlite3_bind_int64(stmt, c + 1, value); - break; - } - case Type::FLOAT: { - double value = reinterpret_cast(*holder).value; - sqlite3_bind_double(stmt, c + 1, value); - break; - } - case Type::STRING: { - std::shared_ptr buffer = - reinterpret_cast(*holder).value; - const std::string string = buffer->ToString(); - const char* value = string.c_str(); - sqlite3_bind_text(stmt, c + 1, value, static_cast(strlen(value)), - SQLITE_TRANSIENT); - break; - } - case Type::BINARY: { - std::shared_ptr buffer = - reinterpret_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()); - } - } - } - } + int64_t record_count; + ARROW_RETURN_NOT_OK(statement->ExecuteUpdate(&record_count)); + + pb::sql::DoPutUpdateResult result; + result.set_record_count(record_count); + + const std::shared_ptr& buffer = Buffer::FromString(result.SerializeAsString()); + ARROW_RETURN_NOT_OK(writer->WriteMetadata(*buffer)); return Status::OK(); } diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index f5fcdb04143..c877eb16041 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -97,10 +97,14 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { Status DoGetPreparedStatement(const pb::sql::CommandPreparedStatementQuery& command, const ServerCallContext& context, std::unique_ptr* result) override; - Status DoPutPreparedStatement(const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, - std::unique_ptr& reader, - std::unique_ptr& writer) override; + Status DoPutPreparedStatementQuery( + const pb::sql::CommandPreparedStatementQuery& command, + const ServerCallContext& context, std::unique_ptr& reader, + std::unique_ptr& writer) override; + Status DoPutPreparedStatementUpdate( + const pb::sql::CommandPreparedStatementUpdate& command, + const ServerCallContext& context, std::unique_ptr& reader, + std::unique_ptr& writer) override; Status GetFlightInfoTables(const pb::sql::CommandGetTables& command, const ServerCallContext& context, @@ -143,6 +147,8 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { sqlite3* db_; boost::uuids::random_generator uuid_generator_; std::map> prepared_statements_; + + Status GetStatementByHandle(const std::string& prepared_statement_handle, std::shared_ptr* result); }; } // namespace example diff --git a/cpp/src/arrow/flight/flight-sql/server.cpp b/cpp/src/arrow/flight/flight-sql/server.cpp index e83aa61e198..8ef166093cf 100644 --- a/cpp/src/arrow/flight/flight-sql/server.cpp +++ b/cpp/src/arrow/flight/flight-sql/server.cpp @@ -143,7 +143,11 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, } else if (any.Is()) { pb::sql::CommandPreparedStatementQuery command; any.UnpackTo(&command); - return DoPutPreparedStatement(command, context, reader, writer); + return DoPutPreparedStatementQuery(command, context, reader, writer); + } else if (any.Is()) { + pb::sql::CommandPreparedStatementUpdate command; + any.UnpackTo(&command); + return DoPutPreparedStatementUpdate(command, context, reader, writer); } return Status::Invalid("The defined request is invalid."); @@ -312,11 +316,18 @@ Status FlightSqlServerBase::ClosePreparedStatement( return Status::NotImplemented("ClosePreparedStatement not implemented"); } -Status FlightSqlServerBase::DoPutPreparedStatement( +Status FlightSqlServerBase::DoPutPreparedStatementQuery( const pb::sql::CommandPreparedStatementQuery& command, const ServerCallContext& context, std::unique_ptr& reader, std::unique_ptr& writer) { - return Status::NotImplemented("DoPutPreparedStatement not implemented"); + return Status::NotImplemented("DoPutPreparedStatementQuery not implemented"); +} + +Status FlightSqlServerBase::DoPutPreparedStatementUpdate( + const pb::sql::CommandPreparedStatementUpdate& command, + const ServerCallContext& context, std::unique_ptr& reader, + std::unique_ptr& writer) { + return Status::NotImplemented("DoPutPreparedStatementUpdate not implemented"); } Status FlightSqlServerBase::DoPutCommandStatementUpdate( diff --git a/cpp/src/arrow/flight/flight-sql/server.h b/cpp/src/arrow/flight/flight-sql/server.h index 2a1c56f3538..4e8e3da2be4 100644 --- a/cpp/src/arrow/flight/flight-sql/server.h +++ b/cpp/src/arrow/flight/flight-sql/server.h @@ -315,10 +315,22 @@ class FlightSqlServerBase : public FlightServerBase { /// \param[in] context The call context. /// \param[in] reader A sequence of uploaded record batches. /// \param[in] writer Send metadata back to the client. - virtual Status DoPutPreparedStatement( + virtual Status DoPutPreparedStatementQuery( const pb::sql::CommandPreparedStatementQuery& command, const ServerCallContext& context, std::unique_ptr& reader, std::unique_ptr& writer); + + /// \brief Executes an update SQL prepared statement. + /// \param[in] command The CommandPreparedStatementUpdate object containing the + /// prepared statement handle. + /// \param[in] context The call context. + /// \param[in] reader a sequence of uploaded record batches. + /// \param[in] writer send metadata back to the client. + /// \return Status. + virtual Status DoPutPreparedStatementUpdate( + const pb::sql::CommandPreparedStatementUpdate& command, + const ServerCallContext& context, std::unique_ptr& reader, + std::unique_ptr& writer); }; /// \brief Auxiliary class containing all Schemas used on Flight SQL. diff --git a/cpp/src/arrow/flight/flight-sql/server_test.cc b/cpp/src/arrow/flight/flight-sql/server_test.cc index 482cd538a9e..26f3ad6b107 100644 --- a/cpp/src/arrow/flight/flight-sql/server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/server_test.cc @@ -424,6 +424,105 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) ASSERT_TRUE(expected_table->Equals(*table)); } +Status ExecuteCountQuery(const std::string& query, int64_t* result) { + std::unique_ptr flight_info; + ARROW_RETURN_NOT_OK(sql_client->Execute({}, query, &flight_info)); + + std::unique_ptr stream; + ARROW_RETURN_NOT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + 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)); + + *result = reinterpret_cast(*count_scalar).value; + + return Status::OK(); +} + +TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding) { + std::shared_ptr prepared_statement; + ASSERT_OK(sql_client->Prepare( + {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', ?)", + &prepared_statement)); + + std::shared_ptr parameter_schema; + ASSERT_OK(prepared_statement->GetParameterSchema(¶meter_schema)); + + const std::shared_ptr& expected_parameter_schema = + arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); + + ASSERT_TRUE(expected_parameter_schema->Equals(*parameter_schema)); + + std::shared_ptr type_ids; + ArrayFromVector({2}, &type_ids); + std::shared_ptr offsets; + ArrayFromVector({0}, &offsets); + + std::shared_ptr string_array; + ArrayFromVector({}, &string_array); + std::shared_ptr bytes_array; + ArrayFromVector({}, &bytes_array); + std::shared_ptr bigint_array; + ArrayFromVector({999}, &bigint_array); + std::shared_ptr double_array; + ArrayFromVector({}, &double_array); + + 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)); + + int64_t result; + ASSERT_OK(ExecuteCountQuery("SELECT COUNT(*) FROM intTable", &result)); + ASSERT_EQ(3, result); + + ASSERT_OK(prepared_statement->ExecuteUpdate(&result)); + ASSERT_EQ(1, result); + + ASSERT_OK(ExecuteCountQuery("SELECT COUNT(*) FROM intTable", &result)); + ASSERT_EQ(4, result); + + ASSERT_OK(sql_client->ExecuteUpdate( + {}, "DELETE FROM intTable WHERE keyName = 'new_value'", &result)); + ASSERT_EQ(1, result); + + ASSERT_OK(ExecuteCountQuery("SELECT COUNT(*) FROM intTable", &result)); + ASSERT_EQ(3, result); +} + +TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { + std::shared_ptr prepared_statement; + ASSERT_OK(sql_client->Prepare( + {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', 999)", + &prepared_statement)); + + int64_t result; + ASSERT_OK(ExecuteCountQuery("SELECT COUNT(*) FROM intTable", &result)); + ASSERT_EQ(3, result); + + ASSERT_OK(prepared_statement->ExecuteUpdate(&result)); + ASSERT_EQ(1, result); + + ASSERT_OK(ExecuteCountQuery("SELECT COUNT(*) FROM intTable", &result)); + ASSERT_EQ(4, result); + + ASSERT_OK(sql_client->ExecuteUpdate( + {}, "DELETE FROM intTable WHERE keyName = 'new_value'", &result)); + ASSERT_EQ(1, result); + + ASSERT_OK(ExecuteCountQuery("SELECT COUNT(*) FROM intTable", &result)); + ASSERT_EQ(3, result); +} + TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { std::unique_ptr flight_info; std::vector table_types; From e9aafe7c3e9dafa348bf9efea9bc8e2dfd3d091f Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 19 Oct 2021 11:09:31 -0300 Subject: [PATCH 152/237] [C++] CommandGetCrossReference (#172) * Add CommandGetCrossReference on FlightSql.proto * Implement CommandGetCrossReference on C++ client * Implement CommandGetCrossReference on C++ server example * Update FlightSql.proto * Update FlightSql.proto --- cpp/src/arrow/flight/flight-sql/client.h | 18 + cpp/src/arrow/flight/flight-sql/client_impl.h | 27 + .../arrow/flight/flight-sql/client_test.cc | 30 + .../flight-sql/example/sqlite_server.cc | 40 +- .../flight/flight-sql/example/sqlite_server.h | 7 + cpp/src/arrow/flight/flight-sql/server.cpp | 34 +- cpp/src/arrow/flight/flight-sql/server.h | 37 +- .../arrow/flight/flight-sql/server_test.cc | 37 +- format/FlightSql.proto | 1299 +++++++++++++++++ 9 files changed, 1517 insertions(+), 12 deletions(-) create mode 100644 format/FlightSql.proto diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index e7fa511ea2a..6fb94202d17 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -192,6 +192,24 @@ class FlightSqlClientT { const std::string* schema, const std::string& table, std::unique_ptr* flight_info) const; + /// \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_catalog The catalog of the table that exports the key. + /// \param[in] pk_schema The schema of the table that exports the key. + /// \param[in] pk_table The table that exports the key. + /// \param[in] fk_catalog The catalog of the table that imports the key. + /// \param[in] fk_schema The schema of the table that imports the key. + /// \param[in] fk_table The table that imports the key. + /// \param[out] flight_info The FlightInfo describing where to access the dataset. + /// \return Status. + Status GetCrossReference( + const FlightCallOptions& options, const std::string* pk_catalog, + const std::string* pk_schema, const std::string& pk_table, + const std::string* fk_catalog, const std::string* fk_schema, + const std::string& fk_table, std::unique_ptr* flight_info) const; + /// \brief Request a list of table types. /// \param[in] options RPC-layer hints for this call. /// \param[out] flight_info The FlightInfo describing where to access the dataset. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index f7983110434..d325c157245 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -230,6 +230,33 @@ Status FlightSqlClientT::GetImportedKeys( return GetFlightInfoForCommand(client, options, flight_info, command); } +template +Status FlightSqlClientT::GetCrossReference( + const FlightCallOptions& options, const std::string* pk_catalog, + const std::string* pk_schema, const std::string& pk_table, + const std::string* fk_catalog, const std::string* fk_schema, + const std::string& fk_table, std::unique_ptr* flight_info) const { + pb::sql::CommandGetCrossReference command; + + if (pk_catalog != NULLPTR) { + command.set_pk_catalog(*pk_catalog); + } + if (pk_schema != NULLPTR) { + command.set_pk_schema(*pk_schema); + } + command.set_pk_table(pk_table); + + if (fk_catalog != NULLPTR) { + command.set_fk_catalog(*fk_catalog); + } + if (fk_schema != NULLPTR) { + command.set_fk_schema(*fk_schema); + } + command.set_fk_table(fk_table); + + return GetFlightInfoForCommand(client, options, flight_info, command); +} + template Status FlightSqlClientT::GetTableTypes( const FlightCallOptions& options, std::unique_ptr* flight_info) const { diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 457aa37718e..c55228f55e4 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -250,6 +250,36 @@ TEST(TestFlightSqlClient, TestGetPrimary) { (void)sqlClient.GetPrimaryKeys(call_options, &catalog, &schema, table, &flight_info); } +TEST(TestFlightSqlClient, TestGetCrossReference) { + auto* client_mock = new FlightClientMock(); + std::unique_ptr client_mock_ptr(client_mock); + FlightSqlClientT sqlClient(client_mock_ptr); + FlightCallOptions call_options; + + 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_schema(pk_schema); + command.set_pk_table(pk_table); + command.set_fk_catalog(fk_catalog); + command.set_fk_schema(fk_schema); + command.set_fk_table(fk_table); + FlightDescriptor descriptor = getDescriptor(command); + + std::unique_ptr flight_info; + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + + (void)sqlClient.GetCrossReference(call_options, &pk_catalog, &pk_schema, + pk_table, &fk_catalog, &fk_schema, + fk_table, &flight_info); +} + TEST(TestFlightSqlClient, TestExecute) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index cc6933fe8e1..a5fadeb764b 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -633,7 +633,7 @@ Status SQLiteFlightSqlServer::GetFlightInfoImportedKeys( const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { return GetFlightInfoForCommand(descriptor, info, command, - SqlSchema::GetImportedAndExportedKeysSchema()); + SqlSchema::GetImportedKeysSchema()); } Status SQLiteFlightSqlServer::DoGetImportedKeys( @@ -648,15 +648,14 @@ Status SQLiteFlightSqlServer::DoGetImportedKeys( } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); - return DoGetSQLiteQuery(db_, query, SqlSchema::GetImportedAndExportedKeysSchema(), - result); + return DoGetSQLiteQuery(db_, query, SqlSchema::GetImportedKeysSchema(), result); } Status SQLiteFlightSqlServer::GetFlightInfoExportedKeys( const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { return GetFlightInfoForCommand(descriptor, info, command, - SqlSchema::GetImportedAndExportedKeysSchema()); + SqlSchema::GetExportedKeysSchema()); } Status SQLiteFlightSqlServer::DoGetExportedKeys( @@ -671,8 +670,37 @@ Status SQLiteFlightSqlServer::DoGetExportedKeys( } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); - return DoGetSQLiteQuery(db_, query, SqlSchema::GetImportedAndExportedKeysSchema(), - result); + return DoGetSQLiteQuery(db_, query, SqlSchema::GetExportedKeysSchema(), result); +} + +Status SQLiteFlightSqlServer::GetFlightInfoCrossReference( + const pb::sql::CommandGetCrossReference& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return GetFlightInfoForCommand(descriptor, info, command, + SqlSchema::GetCrossReferenceSchema()); +} + +Status SQLiteFlightSqlServer::DoGetCrossReference( + const pb::sql::CommandGetCrossReference& command, const ServerCallContext& context, + std::unique_ptr* result) { + std::string filter = "pk_table_name = '" + command.pk_table() + "'"; + if (command.has_pk_catalog()) { + filter += " AND pk_catalog_name = '" + command.pk_catalog() + "'"; + } + if (command.has_pk_schema()) { + filter += " AND pk_schema_name = '" + command.pk_schema() + "'"; + } + + filter += " AND fk_table_name = '" + command.fk_table() + "'"; + if (command.has_fk_catalog()) { + filter += " AND fk_catalog_name = '" + command.fk_catalog() + "'"; + } + if (command.has_fk_schema()) { + filter += " AND fk_schema_name = '" + command.fk_schema() + "'"; + } + std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); + + return DoGetSQLiteQuery(db_, query, SqlSchema::GetCrossReferenceSchema(), result); } } // namespace example diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index c877eb16041..6dcf4d773da 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -133,6 +133,13 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { Status DoGetExportedKeys(const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, std::unique_ptr* result) override; + Status GetFlightInfoCrossReference(const pb::sql::CommandGetCrossReference& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) override; + Status DoGetCrossReference(const pb::sql::CommandGetCrossReference& command, + const ServerCallContext& context, + std::unique_ptr* result) override; Status GetFlightInfoPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, diff --git a/cpp/src/arrow/flight/flight-sql/server.cpp b/cpp/src/arrow/flight/flight-sql/server.cpp index 8ef166093cf..17b2ec902da 100644 --- a/cpp/src/arrow/flight/flight-sql/server.cpp +++ b/cpp/src/arrow/flight/flight-sql/server.cpp @@ -71,6 +71,10 @@ Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, pb::sql::CommandGetImportedKeys command; any.UnpackTo(&command); return GetFlightInfoImportedKeys(command, context, request, info); + } else if (any.Is()) { + pb::sql::CommandGetCrossReference command; + any.UnpackTo(&command); + return GetFlightInfoCrossReference(command, context, request, info); } return Status::Invalid("The defined request is invalid."); @@ -123,6 +127,10 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket pb::sql::CommandGetImportedKeys command; anyCommand.UnpackTo(&command); return DoGetImportedKeys(command, context, stream); + } else if (anyCommand.Is()) { + pb::sql::CommandGetCrossReference command; + anyCommand.UnpackTo(&command); + return DoGetCrossReference(command, context, stream); } return Status::Invalid("The defined request is invalid."); @@ -304,6 +312,18 @@ Status FlightSqlServerBase::DoGetImportedKeys( return Status::NotImplemented("DoGetImportedKeys not implemented"); } +Status FlightSqlServerBase::GetFlightInfoCrossReference( + const pb::sql::CommandGetCrossReference& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + return Status::NotImplemented("GetFlightInfoCrossReference not implemented"); +} + +Status FlightSqlServerBase::DoGetCrossReference( + const pb::sql::CommandGetCrossReference& command, const ServerCallContext& context, + std::unique_ptr* result) { + return Status::NotImplemented("DoGetCrossReference not implemented"); +} + Status FlightSqlServerBase::CreatePreparedStatement( const pb::sql::ActionCreatePreparedStatementRequest& request, const ServerCallContext& context, std::unique_ptr* p_ptr) { @@ -367,7 +387,7 @@ std::shared_ptr SqlSchema::GetPrimaryKeysSchema() { field("key_sequence", int64()), field("key_name", utf8())}); } -std::shared_ptr SqlSchema::GetImportedAndExportedKeysSchema() { +std::shared_ptr GetImportedExportedKeysAndCrossReferenceSchema() { return arrow::schema( {field("pk_catalog_name", utf8(), true), field("pk_schema_name", utf8(), true), field("pk_table_name", utf8(), false), field("pk_column_name", utf8(), false), @@ -378,6 +398,18 @@ std::shared_ptr SqlSchema::GetImportedAndExportedKeysSchema() { 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(); +} + } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/server.h b/cpp/src/arrow/flight/flight-sql/server.h index 4e8e3da2be4..70612a996ea 100644 --- a/cpp/src/arrow/flight/flight-sql/server.h +++ b/cpp/src/arrow/flight/flight-sql/server.h @@ -280,6 +280,30 @@ class FlightSqlServerBase : public FlightServerBase { const ServerCallContext& context, std::unique_ptr* result); + /// \brief Gets a FlightInfo to extract information about foreign and primary keys. + /// \param[in] command The CommandGetCrossReference object with necessary information + /// to execute the request. + /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. + /// \param[out] info The FlightInfo describing where to access the + /// dataset. + /// \return Status. + virtual Status GetFlightInfoCrossReference(const pb::sql::CommandGetCrossReference& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info); + + /// \brief Gets a FlightDataStream containing the data related to the foreign and + /// primary keys. + /// \param[in] command The CommandGetCrossReference object with necessary information + /// to execute the request. + /// \param[in] context Per-call context. + /// \param[out] result The FlightDataStream containing the results. + /// \return Status. + virtual Status DoGetCrossReference(const pb::sql::CommandGetCrossReference& command, + const ServerCallContext& context, + std::unique_ptr* result); + /// \brief Executes an update SQL statement. /// \param[in] command The CommandStatementUpdate object containing the SQL statement. /// \param[in] context The call context. @@ -363,10 +387,17 @@ class SqlSchema { /// \return The default schema template. static std::shared_ptr GetPrimaryKeysSchema(); - /// \brief Gets the Schema used on CommandGetImportedKeys and CommandGetExportedKeys - /// response. + /// \brief Gets the Schema used on CommandGetImportedKeys response. + /// \return The default schema template. + static std::shared_ptr GetExportedKeysSchema(); + + /// \brief Gets the Schema used on CommandGetImportedKeys response. + /// \return The default schema template. + static std::shared_ptr GetImportedKeysSchema(); + + /// \brief Gets the Schema used on CommandGetCrossReference response. /// \return The default schema template. - static std::shared_ptr GetImportedAndExportedKeysSchema(); + static std::shared_ptr GetCrossReferenceSchema(); }; } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight-sql/server_test.cc b/cpp/src/arrow/flight/flight-sql/server_test.cc index 26f3ad6b107..6a06a1dfaca 100644 --- a/cpp/src/arrow/flight/flight-sql/server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/server_test.cc @@ -573,7 +573,7 @@ TEST(TestFlightSqlServer, TestCommandGetImportedKeys) { DECLARE_ARRAY(delete_rule, UInt8, ({3})); const std::shared_ptr
& expected_table = - Table::Make(SqlSchema::GetImportedAndExportedKeysSchema(), + 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}); @@ -606,7 +606,40 @@ TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { DECLARE_ARRAY(delete_rule, UInt8, ({3})); const std::shared_ptr
& expected_table = - Table::Make(SqlSchema::GetImportedAndExportedKeysSchema(), + 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}); + ASSERT_TRUE(expected_table->Equals(*table)); +} + +TEST(TestFlightSqlServer, TestCommandGetCrossReference) { + std::unique_ptr flight_info; + ASSERT_OK( + sql_client->GetCrossReference({}, NULLPTR, NULLPTR, "foreignTable", NULLPTR, NULLPTR, "intTable", &flight_info)); + + std::unique_ptr stream; + ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + + std::shared_ptr
table; + ASSERT_OK(stream->ReadAll(&table)); + + DECLARE_NULL_ARRAY(pk_catalog_name, String, 1); + DECLARE_NULL_ARRAY(pk_schema_name, String, 1); + DECLARE_ARRAY(pk_table_name, String, ({"foreignTable"})); + DECLARE_ARRAY(pk_column_name, String, ({"id"})); + DECLARE_NULL_ARRAY(fk_catalog_name, String, 1); + DECLARE_NULL_ARRAY(fk_schema_name, String, 1); + DECLARE_ARRAY(fk_table_name, String, ({"intTable"})); + DECLARE_ARRAY(fk_column_name, String, ({"foreignId"})); + DECLARE_ARRAY(key_sequence, Int32, ({0})); + DECLARE_NULL_ARRAY(fk_key_name, String, 1); + DECLARE_NULL_ARRAY(pk_key_name, String, 1); + DECLARE_ARRAY(update_rule, UInt8, ({3})); + DECLARE_ARRAY(delete_rule, UInt8, ({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}); diff --git a/format/FlightSql.proto b/format/FlightSql.proto new file mode 100644 index 00000000000..123558c6aa5 --- /dev/null +++ b/format/FlightSql.proto @@ -0,0 +1,1299 @@ +/* + * 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 schema of the query. + * - GetFlightInfo: execute the metadata request. + * + * The returned 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 + * > + * 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 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 UTF-8 string list with values of the supported conversion functions. + SQL_SUPPORTED_CONVERSION_FUNCTIONS = 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 JOINs. + * + * Returns a uint3 uint32 ordinal for the SQL ordering being used, as described in + * `arrow.flight.protocol.sql.SqlJoinsSupportLevel`. + */ + SQL_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; + 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 SqlJoinsSupportLevel { + SQL_JOINS_UNSUPPORTED = 0; + SQL_LIMITED_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; +} + +/* + * Represents a request to retrieve the list of catalogs on a Flight SQL enabled backend. + * Used in the command member of FlightDescriptor for the following RPC calls: + * - GetSchema: return the schema of the query. + * - GetFlightInfo: execute the catalog metadata request. + * + * The returned 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 schemas on a Flight SQL enabled backend. + * Used in the command member of FlightDescriptor for the following RPC calls: + * - GetSchema: return the schema of the query. + * - GetFlightInfo: execute the catalog metadata request. + * + * The returned schema will be: + * < + * catalog_name: utf8, + * schema_name: utf8 not null + * > + * The returned data should be ordered by catalog_name, then schema_name. + */ +message CommandGetSchemas { + 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 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 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 schema of the query. + * - GetFlightInfo: execute the catalog metadata request. + * + * The returned schema will be: + * < + * catalog_name: utf8, + * 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, 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 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 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. + repeated string table_types = 4; + + // Specifies if the 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. + * Used in the command member of FlightDescriptor for the following RPC calls: + * - GetSchema: return the schema of the query. + * - GetFlightInfo: execute the catalog metadata request. + * + * The returned 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 schema of the query. + * - GetFlightInfo: execute the catalog metadata request. + * + * The returned schema will be: + * < + * catalog_name: utf8, + * 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, 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 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 schema of the query. + * - GetFlightInfo: execute the catalog metadata request. + * + * The returned schema will be: + * < + * pk_catalog_name: utf8, + * pk_schema_name: utf8, + * pk_table_name: utf8 not null, + * pk_column_name: utf8 not null, + * fk_catalog_name: utf8, + * fk_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_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 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 schema of the query. + * - GetFlightInfo: execute the catalog metadata request. + * + * The returned schema will be: + * < + * pk_catalog_name: utf8, + * pk_schema_name: utf8, + * pk_table_name: utf8 not null, + * pk_column_name: utf8 not null, + * fk_catalog_name: utf8, + * fk_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_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 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 schema of the query. + * - GetFlightInfo: execute the catalog metadata request. + * + * The returned schema will be: + * < + * pk_catalog_name: utf8, + * pk_schema_name: utf8, + * pk_table_name: utf8 not null, + * pk_column_name: utf8 not null, + * fk_catalog_name: utf8, + * fk_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_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_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_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 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 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; +} From 8b5324f730ca55cc556b3e9517ca40af7826dac6 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 19 Oct 2021 11:18:17 -0300 Subject: [PATCH 153/237] Rename flight-sql/server.cpp to server.cc --- cpp/src/arrow/flight/flight-sql/CMakeLists.txt | 4 ++-- cpp/src/arrow/flight/flight-sql/{server.cpp => server.cc} | 0 2 files changed, 2 insertions(+), 2 deletions(-) rename cpp/src/arrow/flight/flight-sql/{server.cpp => server.cc} (100%) diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 965aeae9305..4426e90bd0c 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -46,7 +46,7 @@ set(CMAKE_CXX_FLAGS_BACKUP "${CMAKE_CXX_FLAGS}") string(REPLACE "/WX" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") string(REPLACE "-Werror " " " CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") -set(ARROW_FLIGHT_SQL_SRCS server.cpp) +set(ARROW_FLIGHT_SQL_SRCS server.cc) add_arrow_lib(arrow_flight_sql CMAKE_PACKAGE_NAME @@ -100,7 +100,7 @@ find_package(SQLite3) include_directories(${SQLite3_INCLUDE_DIRS}) add_executable(flight_sql_test_server - server.cpp + server.cc test_server_cli.cc example/sqlite_statement.cc example/sqlite_statement_batch_reader.cc diff --git a/cpp/src/arrow/flight/flight-sql/server.cpp b/cpp/src/arrow/flight/flight-sql/server.cc similarity index 100% rename from cpp/src/arrow/flight/flight-sql/server.cpp rename to cpp/src/arrow/flight/flight-sql/server.cc From 87e02ac0921ff4277adf8dae10347f222c9b0c37 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 21 Oct 2021 14:06:38 -0300 Subject: [PATCH 154/237] Fix linter issues --- cpp/src/arrow/flight/flight-sql/client.h | 10 +++++----- cpp/src/arrow/flight/flight-sql/client_test.cc | 5 ++--- .../arrow/flight/flight-sql/example/sqlite_server.cc | 3 ++- .../arrow/flight/flight-sql/example/sqlite_server.h | 3 ++- cpp/src/arrow/flight/flight-sql/server.h | 10 +++++----- cpp/src/arrow/flight/flight-sql/server_test.cc | 4 ++-- 6 files changed, 18 insertions(+), 17 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 6fb94202d17..d5401743426 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -204,11 +204,11 @@ class FlightSqlClientT { /// \param[in] fk_table The table that imports the key. /// \param[out] flight_info The FlightInfo describing where to access the dataset. /// \return Status. - Status GetCrossReference( - const FlightCallOptions& options, const std::string* pk_catalog, - const std::string* pk_schema, const std::string& pk_table, - const std::string* fk_catalog, const std::string* fk_schema, - const std::string& fk_table, std::unique_ptr* flight_info) const; + Status GetCrossReference(const FlightCallOptions& options, + const std::string* pk_catalog, const std::string* pk_schema, + const std::string& pk_table, const std::string* fk_catalog, + const std::string* fk_schema, const std::string& fk_table, + std::unique_ptr* flight_info) const; /// \brief Request a list of table types. /// \param[in] options RPC-layer hints for this call. diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index c55228f55e4..3123f5a6512 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -275,9 +275,8 @@ TEST(TestFlightSqlClient, TestGetCrossReference) { std::unique_ptr flight_info; EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void)sqlClient.GetCrossReference(call_options, &pk_catalog, &pk_schema, - pk_table, &fk_catalog, &fk_schema, - fk_table, &flight_info); + (void)sqlClient.GetCrossReference(call_options, &pk_catalog, &pk_schema, pk_table, + &fk_catalog, &fk_schema, fk_table, &flight_info); } TEST(TestFlightSqlClient, TestExecute) { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index a5fadeb764b..844b1e80538 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -496,7 +496,8 @@ Status SQLiteFlightSqlServer::DoGetPreparedStatement( } Status SQLiteFlightSqlServer::GetStatementByHandle( - const std::string& prepared_statement_handle, std::shared_ptr* result) { + const std::string& prepared_statement_handle, + std::shared_ptr* result) { const auto& uuid = boost::lexical_cast(prepared_statement_handle); auto search = prepared_statements_.find(uuid); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index 6dcf4d773da..d3890c6dba7 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -155,7 +155,8 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { boost::uuids::random_generator uuid_generator_; std::map> prepared_statements_; - Status GetStatementByHandle(const std::string& prepared_statement_handle, std::shared_ptr* result); + Status GetStatementByHandle(const std::string& prepared_statement_handle, + std::shared_ptr* result); }; } // namespace example diff --git a/cpp/src/arrow/flight/flight-sql/server.h b/cpp/src/arrow/flight/flight-sql/server.h index 70612a996ea..f48fc7aae14 100644 --- a/cpp/src/arrow/flight/flight-sql/server.h +++ b/cpp/src/arrow/flight/flight-sql/server.h @@ -281,17 +281,17 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about foreign and primary keys. - /// \param[in] command The CommandGetCrossReference object with necessary information + /// \param[in] command The CommandGetCrossReference object with necessary + /// information /// to execute the request. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoCrossReference(const pb::sql::CommandGetCrossReference& command, - const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual Status GetFlightInfoCrossReference( + const pb::sql::CommandGetCrossReference& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the data related to the foreign and /// primary keys. diff --git a/cpp/src/arrow/flight/flight-sql/server_test.cc b/cpp/src/arrow/flight/flight-sql/server_test.cc index 6a06a1dfaca..fdb5a368edc 100644 --- a/cpp/src/arrow/flight/flight-sql/server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/server_test.cc @@ -615,8 +615,8 @@ TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { TEST(TestFlightSqlServer, TestCommandGetCrossReference) { std::unique_ptr flight_info; - ASSERT_OK( - sql_client->GetCrossReference({}, NULLPTR, NULLPTR, "foreignTable", NULLPTR, NULLPTR, "intTable", &flight_info)); + ASSERT_OK(sql_client->GetCrossReference({}, NULLPTR, NULLPTR, "foreignTable", NULLPTR, + NULLPTR, "intTable", &flight_info)); std::unique_ptr stream; ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); From 655d8dd0b88ccfc0905226dc6fa1058c9917f22c Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 22 Oct 2021 16:26:19 -0300 Subject: [PATCH 155/237] Use ASSERT_OK on client_test.cc --- .../arrow/flight/flight-sql/client_test.cc | 33 ++++++++++--------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 3123f5a6512..165527a141d 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -115,7 +115,7 @@ TEST(TestFlightSqlClient, TestGetCatalogs) { std::unique_ptr flight_info; EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void)sqlClient.GetCatalogs(call_options, &flight_info); + ASSERT_OK(sqlClient.GetCatalogs(call_options, &flight_info)); } TEST(TestFlightSqlClient, TestGetSchemas) { @@ -135,8 +135,8 @@ TEST(TestFlightSqlClient, TestGetSchemas) { std::unique_ptr flight_info; EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void)sqlClient.GetSchemas(call_options, &catalog, &schema_filter_pattern, - &flight_info); + ASSERT_OK(sqlClient.GetSchemas(call_options, &catalog, &schema_filter_pattern, + &flight_info)); } TEST(TestFlightSqlClient, TestGetTables) { @@ -164,9 +164,9 @@ TEST(TestFlightSqlClient, TestGetTables) { std::unique_ptr flight_info; EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void)sqlClient.GetTables(call_options, &catalog, &schema_filter_pattern, + ASSERT_OK(sqlClient.GetTables(call_options, &catalog, &schema_filter_pattern, &table_name_filter_pattern, include_schema, table_types, - &flight_info); + &flight_info)); } TEST(TestFlightSqlClient, TestGetTableTypes) { @@ -181,7 +181,7 @@ TEST(TestFlightSqlClient, TestGetTableTypes) { std::unique_ptr flight_info; EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void)sqlClient.GetTableTypes(call_options, &flight_info); + ASSERT_OK(sqlClient.GetTableTypes(call_options, &flight_info)); } TEST(TestFlightSqlClient, TestGetExported) { @@ -203,7 +203,7 @@ TEST(TestFlightSqlClient, TestGetExported) { std::unique_ptr flight_info; EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void)sqlClient.GetExportedKeys(call_options, &catalog, &schema, table, &flight_info); + ASSERT_OK(sqlClient.GetExportedKeys(call_options, &catalog, &schema, table, &flight_info)); } TEST(TestFlightSqlClient, TestGetImported) { @@ -225,7 +225,7 @@ TEST(TestFlightSqlClient, TestGetImported) { std::unique_ptr flight_info; EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void)sqlClient.GetImportedKeys(call_options, &catalog, &schema, table, &flight_info); + ASSERT_OK(sqlClient.GetImportedKeys(call_options, &catalog, &schema, table, &flight_info)); } TEST(TestFlightSqlClient, TestGetPrimary) { @@ -247,7 +247,7 @@ TEST(TestFlightSqlClient, TestGetPrimary) { std::unique_ptr flight_info; EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void)sqlClient.GetPrimaryKeys(call_options, &catalog, &schema, table, &flight_info); + ASSERT_OK(sqlClient.GetPrimaryKeys(call_options, &catalog, &schema, table, &flight_info)); } TEST(TestFlightSqlClient, TestGetCrossReference) { @@ -275,8 +275,9 @@ TEST(TestFlightSqlClient, TestGetCrossReference) { std::unique_ptr flight_info; EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void)sqlClient.GetCrossReference(call_options, &pk_catalog, &pk_schema, pk_table, - &fk_catalog, &fk_schema, fk_table, &flight_info); + ASSERT_OK(sqlClient.GetCrossReference(call_options, &pk_catalog, &pk_schema, + pk_table, &fk_catalog, &fk_schema, + fk_table, &flight_info)); } TEST(TestFlightSqlClient, TestExecute) { @@ -294,7 +295,7 @@ TEST(TestFlightSqlClient, TestExecute) { std::unique_ptr flight_info; EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void)sqlClient.Execute(call_options, query, &flight_info); + ASSERT_OK(sqlClient.Execute(call_options, query, &flight_info)); } TEST(TestFlightSqlClient, TestPreparedStatementExecute) { @@ -326,11 +327,11 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecute) { EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); std::shared_ptr> preparedStatement; - (void)sqlClient.Prepare(call_options, query, &preparedStatement); + ASSERT_OK(sqlClient.Prepare(call_options, query, &preparedStatement)); EXPECT_CALL(*client_mock, GetFlightInfo(_, _, &flight_info)); - (void)preparedStatement->Execute(&flight_info); + ASSERT_OK(preparedStatement->Execute(&flight_info)); } TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { @@ -446,7 +447,7 @@ TEST(TestFlightSqlClient, TestExecuteUpdate) { int64_t num_rows; - (void)sqlClient.ExecuteUpdate(call_options, query, &num_rows); + ASSERT_OK(sqlClient.ExecuteUpdate(call_options, query, &num_rows)); ASSERT_EQ(num_rows, 100); } @@ -470,7 +471,7 @@ TEST(TestFlightSqlClient, TestGetSqlInfo) { FlightCallOptions call_options; EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - (void)sql_client.GetSqlInfo(call_options, sql_info, &flight_info); + ASSERT_OK(sql_client.GetSqlInfo(call_options, sql_info, &flight_info)); } template From 3895b53c5a49b37ebc403c7aa4f897eb06a85209 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 25 Oct 2021 11:26:31 -0300 Subject: [PATCH 156/237] [C++] Improve Client interface (#180) * Use '#pragma once' on header files * Remove ghost parameter on Doxygen for PreparedStatement.Execute * Change client interface to use Result instead of Status * Rename sqlClient to sql_client on tests --- cpp/src/arrow/flight/flight-sql/client.h | 148 +++++------- cpp/src/arrow/flight/flight-sql/client_impl.h | 152 ++++++------ .../arrow/flight/flight-sql/client_test.cc | 111 ++++----- .../sqlite_tables_schema_batch_reader.h | 5 +- .../arrow/flight/flight-sql/server_test.cc | 223 +++++++++--------- .../arrow/flight/flight-sql/test_app_cli.cc | 62 +++-- 6 files changed, 317 insertions(+), 384 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index d5401743426..d834f9aa2f1 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -15,13 +15,13 @@ // specific language governing permissions and limitations // under the License. -#ifndef ARROW_FLIGHT_SQL_CLIENT_H -#define ARROW_FLIGHT_SQL_CLIENT_H +#pragma once #include #include #include #include +#include #include #include @@ -55,25 +55,20 @@ class ARROW_EXPORT PreparedStatementT { ~PreparedStatementT(); /// \brief Executes the prepared statement query on the server. - /// \param[in] call_options RPC-layer hints for this call. - /// \param[out] info A FlightInfo object representing the stream(s) to fetch. - /// \return Status. - Status Execute(std::unique_ptr* info); + /// \return A FlightInfo object representing the stream(s) to fetch. + arrow::Result> Execute(); /// \brief Executes the prepared statement update query on the server. - /// \param rows[out] The number of rows affected. - /// \return Status. - Status ExecuteUpdate(int64_t* rows); + /// \return The number of rows affected. + arrow::Result ExecuteUpdate(); /// \brief Retrieve the parameter schema from the query. - /// \param schema The parameter schema from the query. - /// \return Status. - Status GetParameterSchema(std::shared_ptr* schema); + /// \return The parameter schema from the query. + arrow::Result> GetParameterSchema(); /// \brief Retrieve the ResultSet schema from the query. - /// \param schema The ResultSet schema from the query. - /// \return Status. - Status GetResultSetSchema(std::shared_ptr* schema); + /// \return The ResultSet schema from the query. + arrow::Result> GetResultSetSchema(); /// \brief Set a RecordBatch that contains the parameters that will be bind. /// \param parameter_binding_ The parameters that will be bind. @@ -101,45 +96,39 @@ class FlightSqlClientT { /// \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. - /// \param[out] flight_info The FlightInfo describing where to access the dataset - /// \return Status. - Status Execute(const FlightCallOptions& options, const std::string& query, - std::unique_ptr* flight_info) const; + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> Execute(const FlightCallOptions& options, + const std::string& query) const; /// \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. - /// \param[out] rows The quantity of rows affected by the operation. - /// \return Status. - Status ExecuteUpdate(const FlightCallOptions& options, const std::string& query, - int64_t* rows) const; + /// \return The quantity of rows affected by the operation. + arrow::Result ExecuteUpdate(const FlightCallOptions& options, + const std::string& query) const; /// \brief Request a list of catalogs. /// \param[in] options RPC-layer hints for this call. - /// \param[out] flight_info The FlightInfo describing where to access the dataset. - /// \return Status. - Status GetCatalogs(const FlightCallOptions& options, - std::unique_ptr* flight_info) const; + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetCatalogs( + const FlightCallOptions& options) const; /// \brief Request a list of schemas. /// \param[in] options RPC-layer hints for this call. /// \param[in] catalog The catalog. /// \param[in] schema_filter_pattern The schema filter pattern. - /// \param[out] flight_info The FlightInfo describing where to access the - /// dataset. - /// \return Status. - Status GetSchemas(const FlightCallOptions& options, const std::string* catalog, - const std::string* schema_filter_pattern, - std::unique_ptr* flight_info) const; + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetSchemas( + const FlightCallOptions& options, const std::string* catalog, + const std::string* schema_filter_pattern) const; /// \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 - /// \param[out] stream the returned RecordBatchReader - /// \return Status - Status DoGet(const FlightCallOptions& options, const Ticket& ticket, - std::unique_ptr* stream) const; + /// \return The returned RecordBatchReader + arrow::Result> DoGet( + const FlightCallOptions& options, const Ticket& ticket) const; /// \brief Request a list of tables. /// \param[in] options RPC-layer hints for this call. @@ -149,25 +138,21 @@ class FlightSqlClientT { /// \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. - /// \param[out] flight_info The FlightInfo describing where to access the - /// dataset. - /// \return Status. - Status GetTables(const FlightCallOptions& options, const std::string* catalog, - const std::string* schema_filter_pattern, - const std::string* table_filter_pattern, bool include_schema, - std::vector& table_types, - std::unique_ptr* flight_info) const; + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetTables( + const FlightCallOptions& options, const std::string* catalog, + const std::string* schema_filter_pattern, const std::string* table_filter_pattern, + bool include_schema, std::vector& table_types) const; /// \brief Request the primary keys for a table. /// \param[in] options RPC-layer hints for this call. /// \param[in] catalog The catalog. /// \param[in] schema The schema. /// \param[in] table The table. - /// \param[out] flight_info The FlightInfo describing where to access the dataset. - /// \return Status. - Status GetPrimaryKeys(const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table, - std::unique_ptr* flight_info) const; + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetPrimaryKeys( + const FlightCallOptions& options, const std::string* catalog, + const std::string* schema, const std::string& table) const; /// \brief Retrieves a description about the foreign key columns that reference the /// primary key columns of the given table. @@ -175,22 +160,20 @@ class FlightSqlClientT { /// \param[in] catalog The foreign key table catalog. /// \param[in] schema The foreign key table schema. /// \param[in] table The foreign key table. Cannot be null. - /// \param[out] flight_info The FlightInfo describing where to access the dataset. - /// \return Status. - Status GetExportedKeys(const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table, - std::unique_ptr* flight_info) const; + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetExportedKeys( + const FlightCallOptions& options, const std::string* catalog, + const std::string* schema, const std::string& table) const; /// \brief Retrieves the foreign key columns for the given table. /// \param[in] options RPC-layer hints for this call. /// \param[in] catalog The primary key table catalog. /// \param[in] schema The primary key table schema. /// \param[in] table The primary key table. Cannot be null. - /// \param[out] flight_info The FlightInfo describing where to access the dataset. - /// \return Status. - Status GetImportedKeys(const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table, - std::unique_ptr* flight_info) const; + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetImportedKeys( + const FlightCallOptions& options, const std::string* catalog, + const std::string* schema, const std::string& table) const; /// \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 @@ -202,45 +185,40 @@ class FlightSqlClientT { /// \param[in] fk_catalog The catalog of the table that imports the key. /// \param[in] fk_schema The schema of the table that imports the key. /// \param[in] fk_table The table that imports the key. - /// \param[out] flight_info The FlightInfo describing where to access the dataset. - /// \return Status. - Status GetCrossReference(const FlightCallOptions& options, - const std::string* pk_catalog, const std::string* pk_schema, - const std::string& pk_table, const std::string* fk_catalog, - const std::string* fk_schema, const std::string& fk_table, - std::unique_ptr* flight_info) const; + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetCrossReference( + const FlightCallOptions& options, const std::string* pk_catalog, + const std::string* pk_schema, const std::string& pk_table, + const std::string* fk_catalog, const std::string* fk_schema, + const std::string& fk_table) const; /// \brief Request a list of table types. /// \param[in] options RPC-layer hints for this call. - /// \param[out] flight_info The FlightInfo describing where to access the dataset. - /// \return Status. - Status GetTableTypes(const FlightCallOptions& options, - std::unique_ptr* flight_info) const; + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetTableTypes( + const FlightCallOptions& options) const; /// \brief Request a list of SQL information. /// \param[in] options RPC-layer hints for this call. /// \param[in] sql_info the SQL info required. - /// \param[out] flight_info The FlightInfo describing where to access the dataset. - /// \return Status. - Status GetSqlInfo(const FlightCallOptions& options, const std::vector& sql_info, - std::unique_ptr* flight_info) const; + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetSqlInfo( + const FlightCallOptions& options, const std::vector& sql_info) const; /// \brief Request a list of SQL information. /// \param[in] options RPC-layer hints for this call. /// \param[in] sql_info the SQL info required. - /// \param[out] flight_info The FlightInfo describing where to access the dataset. - /// \return Status. - Status GetSqlInfo(const FlightCallOptions& options, - const std::vector& sql_info, - std::unique_ptr* flight_info) const; + /// \return The FlightInfo describing where to access the dataset. + arrow::Result> GetSqlInfo( + const FlightCallOptions& options, + const std::vector& sql_info) const; /// \brief Create a prepared statement object. /// \param[in] options RPC-layer hints for this call. /// \param[in] query The query that will be executed. - /// \param[out] prepared_statement The created prepared statement. - /// \return Status. - Status Prepare(const FlightCallOptions& options, const std::string& query, - std::shared_ptr>* prepared_statement); + /// \return The created prepared statement. + arrow::Result>> Prepare( + const FlightCallOptions& options, const std::string& query); private: std::unique_ptr client; @@ -255,6 +233,4 @@ using PreparedStatement = internal::PreparedStatementT<>; } // namespace flight } // namespace arrow -#endif // ARROW_FLIGHT_SQL_CLIENT_H - #include diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index d325c157245..b8d32c33bee 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -67,28 +68,29 @@ inline FlightDescriptor GetFlightDescriptorForCommand( } template -Status GetFlightInfoForCommand(const std::unique_ptr& client, - const FlightCallOptions& options, - std::unique_ptr* flight_info, - const google::protobuf::Message& command) { +arrow::Result> GetFlightInfoForCommand( + const std::unique_ptr& client, const FlightCallOptions& options, + const google::protobuf::Message& command) { const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); - return client->GetFlightInfo(options, descriptor, flight_info); + std::unique_ptr flight_info; + ARROW_RETURN_NOT_OK(client->GetFlightInfo(options, descriptor, &flight_info)); + + return std::move(flight_info); } template -Status FlightSqlClientT::Execute(const FlightCallOptions& options, - const std::string& query, - std::unique_ptr* flight_info) const { +arrow::Result> FlightSqlClientT::Execute( + const FlightCallOptions& options, const std::string& query) const { pb::sql::CommandStatementQuery command; command.set_query(query); - return GetFlightInfoForCommand(client, options, flight_info, command); + return GetFlightInfoForCommand(client, options, command); } template -Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, - const std::string& query, int64_t* rows) const { +arrow::Result FlightSqlClientT::ExecuteUpdate( + const FlightCallOptions& options, const std::string& query) const { pb::sql::CommandStatementUpdate command; command.set_query(query); @@ -110,24 +112,21 @@ Status FlightSqlClientT::ExecuteUpdate(const FlightCallOptions& options, const std::string& string = pBuffer->ToString(); doPutUpdateResult.ParseFrom(string); - *rows = doPutUpdateResult.record_count(); - - return Status::OK(); + return doPutUpdateResult.record_count(); } template -Status FlightSqlClientT::GetCatalogs(const FlightCallOptions& options, - std::unique_ptr* flight_info) const { +arrow::Result> FlightSqlClientT::GetCatalogs( + const FlightCallOptions& options) const { pb::sql::CommandGetCatalogs command; - return GetFlightInfoForCommand(client, options, flight_info, command); + return GetFlightInfoForCommand(client, options, command); } template -Status FlightSqlClientT::GetSchemas(const FlightCallOptions& options, - const std::string* catalog, - const std::string* schema_filter_pattern, - std::unique_ptr* flight_info) const { +arrow::Result> FlightSqlClientT::GetSchemas( + const FlightCallOptions& options, const std::string* catalog, + const std::string* schema_filter_pattern) const { pb::sql::CommandGetSchemas command; if (catalog != NULLPTR) { command.set_catalog(*catalog); @@ -136,17 +135,14 @@ Status FlightSqlClientT::GetSchemas(const FlightCallOptions& options, command.set_schema_filter_pattern(*schema_filter_pattern); } - return GetFlightInfoForCommand(client, options, flight_info, command); + return GetFlightInfoForCommand(client, options, command); } template -Status FlightSqlClientT::GetTables(const FlightCallOptions& options, - const std::string* catalog, - const std::string* schema_filter_pattern, - const std::string* table_filter_pattern, - bool include_schema, - std::vector& table_types, - std::unique_ptr* flight_info) const { +arrow::Result> FlightSqlClientT::GetTables( + const FlightCallOptions& options, const std::string* catalog, + const std::string* schema_filter_pattern, const std::string* table_filter_pattern, + bool include_schema, std::vector& table_types) const { pb::sql::CommandGetTables command; if (catalog != NULLPTR) { @@ -167,14 +163,13 @@ Status FlightSqlClientT::GetTables(const FlightCallOptions& options, command.add_table_types(table_type); } - return GetFlightInfoForCommand(client, options, flight_info, command); + return GetFlightInfoForCommand(client, options, command); } template -Status FlightSqlClientT::GetPrimaryKeys( +arrow::Result> FlightSqlClientT::GetPrimaryKeys( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table, - std::unique_ptr* flight_info) const { + const std::string* schema, const std::string& table) const { pb::sql::CommandGetPrimaryKeys command; if (catalog != NULLPTR) { @@ -187,14 +182,13 @@ Status FlightSqlClientT::GetPrimaryKeys( command.set_table(table); - return GetFlightInfoForCommand(client, options, flight_info, command); + return GetFlightInfoForCommand(client, options, command); } template -Status FlightSqlClientT::GetExportedKeys( +arrow::Result> FlightSqlClientT::GetExportedKeys( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table, - std::unique_ptr* flight_info) const { + const std::string* schema, const std::string& table) const { pb::sql::CommandGetExportedKeys command; if (catalog != NULLPTR) { @@ -207,14 +201,13 @@ Status FlightSqlClientT::GetExportedKeys( command.set_table(table); - return GetFlightInfoForCommand(client, options, flight_info, command); + return GetFlightInfoForCommand(client, options, command); } template -Status FlightSqlClientT::GetImportedKeys( +arrow::Result> FlightSqlClientT::GetImportedKeys( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table, - std::unique_ptr* flight_info) const { + const std::string* schema, const std::string& table) const { pb::sql::CommandGetImportedKeys command; if (catalog != NULLPTR) { @@ -227,15 +220,15 @@ Status FlightSqlClientT::GetImportedKeys( command.set_table(table); - return GetFlightInfoForCommand(client, options, flight_info, command); + return GetFlightInfoForCommand(client, options, command); } template -Status FlightSqlClientT::GetCrossReference( +arrow::Result> FlightSqlClientT::GetCrossReference( const FlightCallOptions& options, const std::string* pk_catalog, const std::string* pk_schema, const std::string& pk_table, const std::string* fk_catalog, const std::string* fk_schema, - const std::string& fk_table, std::unique_ptr* flight_info) const { + const std::string& fk_table) const { pb::sql::CommandGetCrossReference command; if (pk_catalog != NULLPTR) { @@ -254,27 +247,29 @@ Status FlightSqlClientT::GetCrossReference( } command.set_fk_table(fk_table); - return GetFlightInfoForCommand(client, options, flight_info, command); + return GetFlightInfoForCommand(client, options, command); } template -Status FlightSqlClientT::GetTableTypes( - const FlightCallOptions& options, std::unique_ptr* flight_info) const { +arrow::Result> FlightSqlClientT::GetTableTypes( + const FlightCallOptions& options) const { pb::sql::CommandGetTableTypes command; - return GetFlightInfoForCommand(client, options, flight_info, command); + return GetFlightInfoForCommand(client, options, command); } template -Status FlightSqlClientT::DoGet(const FlightCallOptions& options, const Ticket& ticket, - std::unique_ptr* stream) const { - return client->DoGet(options, ticket, stream); +arrow::Result> FlightSqlClientT::DoGet( + const FlightCallOptions& options, const Ticket& ticket) const { + std::unique_ptr stream; + ARROW_RETURN_NOT_OK(client->DoGet(options, ticket, &stream)); + + return std::move(stream); } template -Status FlightSqlClientT::Prepare( - const FlightCallOptions& options, const std::string& query, - std::shared_ptr>* prepared_statement) { +arrow::Result>> FlightSqlClientT::Prepare( + const FlightCallOptions& options, const std::string& query) { google::protobuf::Any command; pb::sql::ActionCreatePreparedStatementRequest request; request.set_query(query); @@ -301,14 +296,12 @@ Status FlightSqlClientT::Prepare( prepared_result.UnpackTo(&prepared_statement_result); - prepared_statement->reset( + return std::shared_ptr>( new PreparedStatementT(client.get(), query, prepared_statement_result, options)); - - return Status::OK(); } template -Status PreparedStatementT::Execute(std::unique_ptr* info) { +arrow::Result> PreparedStatementT::Execute() { if (is_closed) { return Status::Invalid("Statement already closed."); } @@ -336,11 +329,14 @@ Status PreparedStatementT::Execute(std::unique_ptr* info) { ARROW_RETURN_NOT_OK(reader->ReadMetadata(&buffer)); } - return client->GetFlightInfo(options, descriptor, info); + std::unique_ptr info; + ARROW_RETURN_NOT_OK(client->GetFlightInfo(options, descriptor, &info)); + + return std::move(info); } template -Status PreparedStatementT::ExecuteUpdate(int64_t* rows) { +arrow::Result PreparedStatementT::ExecuteUpdate() { if (is_closed) { return Status::Invalid("Statement already closed."); } @@ -367,13 +363,14 @@ Status PreparedStatementT::ExecuteUpdate(int64_t* rows) { ARROW_RETURN_NOT_OK(writer->DoneWriting()); std::shared_ptr metadata; ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata)); + ARROW_RETURN_NOT_OK(writer->Close()); + pb::sql::DoPutUpdateResult doPutUpdateResult; const std::string& metadataAsString = metadata->ToString(); doPutUpdateResult.ParseFrom(metadataAsString); - *rows = doPutUpdateResult.record_count(); - ARROW_RETURN_NOT_OK(writer->Close()); - return Status::OK(); + + return doPutUpdateResult.record_count(); } template @@ -390,29 +387,25 @@ bool PreparedStatementT::IsClosed() const { } template -Status PreparedStatementT::GetResultSetSchema(std::shared_ptr* schema) { +arrow::Result> PreparedStatementT::GetResultSetSchema() { auto& args = prepared_statement_result.dataset_schema(); std::shared_ptr schema_buffer = std::make_shared(args); io::BufferReader reader(schema_buffer); ipc::DictionaryMemo in_memo; - ARROW_ASSIGN_OR_RAISE(*schema, ReadSchema(&reader, &in_memo)) - - return Status::OK(); + return ReadSchema(&reader, &in_memo); } template -Status PreparedStatementT::GetParameterSchema(std::shared_ptr* schema) { +arrow::Result> PreparedStatementT::GetParameterSchema() { auto& args = prepared_statement_result.parameter_schema(); std::shared_ptr schema_buffer = std::make_shared(args); io::BufferReader reader(schema_buffer); ipc::DictionaryMemo in_memo; - ARROW_ASSIGN_OR_RAISE(*schema, ReadSchema(&reader, &in_memo)) - - return Status::OK(); + return ReadSchema(&reader, &in_memo); } template @@ -441,20 +434,19 @@ Status PreparedStatementT::Close() { } template -Status FlightSqlClientT::GetSqlInfo(const FlightCallOptions& options, - const std::vector& sql_info, - std::unique_ptr* flight_info) const { +arrow::Result> FlightSqlClientT::GetSqlInfo( + const FlightCallOptions& options, const std::vector& sql_info) const { pb::sql::CommandGetSqlInfo command; for (const int& info : sql_info) command.add_info(info); - return GetFlightInfoForCommand(client, options, flight_info, command); + + return GetFlightInfoForCommand(client, options, command); } template -Status FlightSqlClientT::GetSqlInfo(const FlightCallOptions& options, - const std::vector& sql_info, - std::unique_ptr* flight_info) const { - return GetSqlInfo(options, reinterpret_cast&>(sql_info), - flight_info); +arrow::Result> FlightSqlClientT::GetSqlInfo( + const FlightCallOptions& options, + const std::vector& sql_info) const { + return GetSqlInfo(options, reinterpret_cast&>(sql_info)); } } // namespace internal diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 165527a141d..77376f704c1 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -106,22 +106,21 @@ FlightDescriptor getDescriptor(google::protobuf::Message& command) { TEST(TestFlightSqlClient, TestGetCatalogs) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sqlClient(client_mock_ptr); + FlightSqlClientT sql_client(client_mock_ptr); FlightCallOptions call_options; pb::sql::CommandGetCatalogs command; FlightDescriptor descriptor = getDescriptor(command); - std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sqlClient.GetCatalogs(call_options, &flight_info)); + ASSERT_OK(sql_client.GetCatalogs(call_options)); } TEST(TestFlightSqlClient, TestGetSchemas) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sqlClient(client_mock_ptr); + FlightSqlClientT sql_client(client_mock_ptr); FlightCallOptions call_options; std::string schema_filter_pattern = "schema_filter_pattern"; @@ -132,17 +131,15 @@ TEST(TestFlightSqlClient, TestGetSchemas) { command.set_schema_filter_pattern(schema_filter_pattern); FlightDescriptor descriptor = getDescriptor(command); - std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sqlClient.GetSchemas(call_options, &catalog, &schema_filter_pattern, - &flight_info)); + ASSERT_OK(sql_client.GetSchemas(call_options, &catalog, &schema_filter_pattern)); } TEST(TestFlightSqlClient, TestGetTables) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sqlClient(client_mock_ptr); + FlightSqlClientT sql_client(client_mock_ptr); FlightCallOptions call_options; std::string catalog = "catalog"; @@ -161,33 +158,31 @@ TEST(TestFlightSqlClient, TestGetTables) { } FlightDescriptor descriptor = getDescriptor(command); - std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sqlClient.GetTables(call_options, &catalog, &schema_filter_pattern, - &table_name_filter_pattern, include_schema, table_types, - &flight_info)); + ASSERT_OK(sql_client.GetTables(call_options, &catalog, &schema_filter_pattern, + &table_name_filter_pattern, include_schema, + table_types)); } TEST(TestFlightSqlClient, TestGetTableTypes) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sqlClient(client_mock_ptr); + FlightSqlClientT sql_client(client_mock_ptr); FlightCallOptions call_options; pb::sql::CommandGetTableTypes command; FlightDescriptor descriptor = getDescriptor(command); - std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sqlClient.GetTableTypes(call_options, &flight_info)); + ASSERT_OK(sql_client.GetTableTypes(call_options)); } TEST(TestFlightSqlClient, TestGetExported) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sqlClient(client_mock_ptr); + FlightSqlClientT sql_client(client_mock_ptr); FlightCallOptions call_options; std::string catalog = "catalog"; @@ -200,16 +195,15 @@ TEST(TestFlightSqlClient, TestGetExported) { command.set_table(table); FlightDescriptor descriptor = getDescriptor(command); - std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sqlClient.GetExportedKeys(call_options, &catalog, &schema, table, &flight_info)); + ASSERT_OK(sql_client.GetExportedKeys(call_options, &catalog, &schema, table)); } TEST(TestFlightSqlClient, TestGetImported) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sqlClient(client_mock_ptr); + FlightSqlClientT sql_client(client_mock_ptr); FlightCallOptions call_options; std::string catalog = "catalog"; @@ -222,16 +216,15 @@ TEST(TestFlightSqlClient, TestGetImported) { command.set_table(table); FlightDescriptor descriptor = getDescriptor(command); - std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sqlClient.GetImportedKeys(call_options, &catalog, &schema, table, &flight_info)); + ASSERT_OK(sql_client.GetImportedKeys(call_options, &catalog, &schema, table)); } TEST(TestFlightSqlClient, TestGetPrimary) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sqlClient(client_mock_ptr); + FlightSqlClientT sql_client(client_mock_ptr); FlightCallOptions call_options; std::string catalog = "catalog"; @@ -244,16 +237,15 @@ TEST(TestFlightSqlClient, TestGetPrimary) { command.set_table(table); FlightDescriptor descriptor = getDescriptor(command); - std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sqlClient.GetPrimaryKeys(call_options, &catalog, &schema, table, &flight_info)); + ASSERT_OK(sql_client.GetPrimaryKeys(call_options, &catalog, &schema, table)); } TEST(TestFlightSqlClient, TestGetCrossReference) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sqlClient(client_mock_ptr); + FlightSqlClientT sql_client(client_mock_ptr); FlightCallOptions call_options; std::string pk_catalog = "pk_catalog"; @@ -272,18 +264,16 @@ TEST(TestFlightSqlClient, TestGetCrossReference) { command.set_fk_table(fk_table); FlightDescriptor descriptor = getDescriptor(command); - std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sqlClient.GetCrossReference(call_options, &pk_catalog, &pk_schema, - pk_table, &fk_catalog, &fk_schema, - fk_table, &flight_info)); + ASSERT_OK(sql_client.GetCrossReference(call_options, &pk_catalog, &pk_schema, pk_table, + &fk_catalog, &fk_schema, fk_table)); } TEST(TestFlightSqlClient, TestExecute) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sqlClient(client_mock_ptr); + FlightSqlClientT sql_client(client_mock_ptr); FlightCallOptions call_options; std::string query = "query"; @@ -292,16 +282,15 @@ TEST(TestFlightSqlClient, TestExecute) { command.set_query(query); FlightDescriptor descriptor = getDescriptor(command); - std::unique_ptr flight_info; - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sqlClient.Execute(call_options, query, &flight_info)); + ASSERT_OK(sql_client.Execute(call_options, query)); } TEST(TestFlightSqlClient, TestPreparedStatementExecute) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sqlClient(client_mock_ptr); + FlightSqlClientT sql_client(client_mock_ptr); FlightCallOptions call_options; const std::string query = "query"; @@ -323,21 +312,19 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecute) { return Status::OK(); }); - std::unique_ptr flight_info; EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); - std::shared_ptr> preparedStatement; - ASSERT_OK(sqlClient.Prepare(call_options, query, &preparedStatement)); + ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client.Prepare(call_options, query)); - EXPECT_CALL(*client_mock, GetFlightInfo(_, _, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(_, _, _)); - ASSERT_OK(preparedStatement->Execute(&flight_info)); + ASSERT_OK(prepared_statement->Execute()); } TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sqlClient(client_mock_ptr); + FlightSqlClientT sql_client(client_mock_ptr); FlightCallOptions call_options; const std::string query = "query"; @@ -382,15 +369,12 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { return Status::OK(); }); - std::unique_ptr flight_info; EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); EXPECT_CALL(*client_mock, DoPut(_, _, _, _, _)); - std::shared_ptr> prepared_statement; - ASSERT_OK(sqlClient.Prepare(call_options, query, &prepared_statement)); + ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client.Prepare(call_options, query)); - std::shared_ptr parameter_schema; - ASSERT_OK(prepared_statement->GetParameterSchema(¶meter_schema)); + ASSERT_OK_AND_ASSIGN(auto parameter_schema, prepared_statement->GetParameterSchema()); arrow::Int64Builder int_builder; ASSERT_OK(int_builder.Append(1)); @@ -400,15 +384,15 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { result = arrow::RecordBatch::Make(parameter_schema, 1, {int_array}); ASSERT_OK(prepared_statement->SetParameters(result)); - EXPECT_CALL(*client_mock, GetFlightInfo(_, _, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(_, _, _)); - ASSERT_OK(prepared_statement->Execute(&flight_info)); + ASSERT_OK(prepared_statement->Execute()); } TEST(TestFlightSqlClient, TestExecuteUpdate) { auto* client_mock = new FlightClientMock(); std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sqlClient(client_mock_ptr); + FlightSqlClientT sql_client(client_mock_ptr); FlightCallOptions call_options; std::string query = "query"; @@ -445,9 +429,7 @@ TEST(TestFlightSqlClient, TestExecuteUpdate) { std::unique_ptr reader; EXPECT_CALL(*client_mock, DoPut(Ref(call_options), descriptor, _, _, _)); - int64_t num_rows; - - ASSERT_OK(sqlClient.ExecuteUpdate(call_options, query, &num_rows)); + ASSERT_OK_AND_ASSIGN(auto num_rows, sql_client.ExecuteUpdate(call_options, query)); ASSERT_EQ(num_rows, 100); } @@ -461,7 +443,6 @@ TEST(TestFlightSqlClient, TestGetSqlInfo) { pb::sql::SqlInfo::FLIGHT_SQL_SERVER_NAME, pb::sql::SqlInfo::FLIGHT_SQL_SERVER_VERSION, pb::sql::SqlInfo::FLIGHT_SQL_SERVER_ARROW_VERSION}; - std::unique_ptr flight_info; pb::sql::CommandGetSqlInfo command; for (const auto& info : sql_info) command.add_info(info); @@ -470,8 +451,8 @@ TEST(TestFlightSqlClient, TestGetSqlInfo) { const FlightDescriptor& descriptor = FlightDescriptor::Command(any.SerializeAsString()); FlightCallOptions call_options; - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, &flight_info)); - ASSERT_OK(sql_client.GetSqlInfo(call_options, sql_info, &flight_info)); + EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); + ASSERT_OK(sql_client.GetSqlInfo(call_options, sql_info)); } template @@ -530,11 +511,9 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( EXPECT_CALL(*client_mock, DoPut(_, _, *schema, _, _)); } - int64_t rows; - std::shared_ptr> prepared_statement; - ASSERT_OK(sql_client.Prepare(call_options, query, &prepared_statement)); + ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client.Prepare(call_options, query)); func(prepared_statement, *client_mock, schema, expected_rows); - ASSERT_OK(prepared_statement->ExecuteUpdate(&rows)); + ASSERT_OK_AND_ASSIGN(auto rows, prepared_statement->ExecuteUpdate()); ASSERT_EQ(expected_rows, rows); } diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h index 66a1038e833..10c210bb76f 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h @@ -15,8 +15,7 @@ // specific language governing permissions and limitations // under the License. -#ifndef ARROW_SQLITE_TABLES_SCHEMA_BATCH_READER_H -#define ARROW_SQLITE_TABLES_SCHEMA_BATCH_READER_H +#pragma once #include #include @@ -56,5 +55,3 @@ class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { } // namespace sql } // namespace flight } // namespace arrow - -#endif // ARROW_SQLITE_TABLES_SCHEMA_BATCH_READER_H diff --git a/cpp/src/arrow/flight/flight-sql/server_test.cc b/cpp/src/arrow/flight/flight-sql/server_test.cc index fdb5a368edc..b557e60e0c2 100644 --- a/cpp/src/arrow/flight/flight-sql/server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/server_test.cc @@ -101,11 +101,11 @@ class TestFlightSqlServer : public ::testing::Environment { }; TEST(TestFlightSqlServer, TestCommandStatementQuery) { - std::unique_ptr flight_info; - ASSERT_OK(sql_client->Execute({}, "SELECT * FROM intTable", &flight_info)); + ASSERT_OK_AND_ASSIGN(auto flight_info, + sql_client->Execute({}, "SELECT * FROM intTable")); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr

table; ASSERT_OK(stream->ReadAll(&table)); @@ -126,13 +126,13 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { } TEST(TestFlightSqlServer, TestCommandGetTables) { - std::unique_ptr flight_info; std::vector table_types; - ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types, - &flight_info)); + ASSERT_OK_AND_ASSIGN( + auto flight_info, + sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types)); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -154,15 +154,15 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { } TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { - std::unique_ptr flight_info; std::vector table_types; std::string table_filter_pattern = "int%"; - ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, false, - table_types, &flight_info)); + ASSERT_OK_AND_ASSIGN(auto flight_info, + sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, + false, table_types)); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -179,14 +179,14 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { } TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { - std::unique_ptr flight_info; std::vector table_types{"index"}; - ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types, - &flight_info)); + ASSERT_OK_AND_ASSIGN( + auto flight_info, + sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types)); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -197,14 +197,14 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { } TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { - std::unique_ptr flight_info; std::vector table_types{"table"}; - ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types, - &flight_info)); + ASSERT_OK_AND_ASSIGN( + auto flight_info, + sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types)); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -226,15 +226,15 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { } TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { - std::unique_ptr flight_info; std::vector table_types; std::string table_filter_pattern = "int%"; - ASSERT_OK(sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, true, - table_types, &flight_info)); + ASSERT_OK_AND_ASSIGN(auto flight_info, + sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, + true, table_types)); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -265,11 +265,10 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { } TEST(TestFlightSqlServer, TestCommandGetCatalogs) { - std::unique_ptr flight_info; - ASSERT_OK(sql_client->GetCatalogs({}, &flight_info)); + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetCatalogs({})); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -281,11 +280,10 @@ TEST(TestFlightSqlServer, TestCommandGetCatalogs) { } TEST(TestFlightSqlServer, TestCommandGetSchemas) { - std::unique_ptr flight_info; - ASSERT_OK(sql_client->GetSchemas({}, NULLPTR, NULLPTR, &flight_info)); + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetSchemas({}, NULLPTR, NULLPTR)); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -297,11 +295,10 @@ TEST(TestFlightSqlServer, TestCommandGetSchemas) { } TEST(TestFlightSqlServer, TestCommandGetTableTypes) { - std::unique_ptr flight_info; - ASSERT_OK(sql_client->GetTableTypes({}, &flight_info)); + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetTableTypes({})); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -315,34 +312,33 @@ TEST(TestFlightSqlServer, TestCommandGetTableTypes) { TEST(TestFlightSqlServer, TestCommandStatementUpdate) { int64_t result; - ASSERT_OK(sql_client->ExecuteUpdate( - {}, - "INSERT INTO intTable (keyName, value) VALUES " - "('KEYNAME1', 1001), ('KEYNAME2', 1002), ('KEYNAME3', 1003)", - &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( - sql_client->ExecuteUpdate({}, - "UPDATE intTable SET keyName = 'KEYNAME1' " - "WHERE keyName = 'KEYNAME2' OR keyName = 'KEYNAME3'", - &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(sql_client->ExecuteUpdate( - {}, "DELETE FROM intTable WHERE keyName = 'KEYNAME1'", &result)); + ASSERT_OK_AND_ASSIGN( + result, + sql_client->ExecuteUpdate({}, "DELETE FROM intTable WHERE keyName = 'KEYNAME1'")); ASSERT_EQ(3, result); } TEST(TestFlightSqlServer, TestCommandPreparedStatementQuery) { - std::shared_ptr prepared_statement; - ASSERT_OK(sql_client->Prepare({}, "SELECT * FROM intTable", &prepared_statement)); + ASSERT_OK_AND_ASSIGN(auto prepared_statement, + sql_client->Prepare({}, "SELECT * FROM intTable")); - std::unique_ptr flight_info; - ASSERT_OK(prepared_statement->Execute(&flight_info)); + ASSERT_OK_AND_ASSIGN(auto flight_info, prepared_statement->Execute()); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -363,12 +359,11 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQuery) { } TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) { - std::shared_ptr prepared_statement; - ASSERT_OK(sql_client->Prepare({}, "SELECT * FROM intTable WHERE keyName LIKE ?", - &prepared_statement)); + ASSERT_OK_AND_ASSIGN( + auto prepared_statement, + sql_client->Prepare({}, "SELECT * FROM intTable WHERE keyName LIKE ?")); - std::shared_ptr parameter_schema; - ASSERT_OK(prepared_statement->GetParameterSchema(¶meter_schema)); + ASSERT_OK_AND_ASSIGN(auto parameter_schema, prepared_statement->GetParameterSchema()); const std::shared_ptr& expected_parameter_schema = arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); @@ -400,11 +395,10 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) ASSERT_OK(prepared_statement->SetParameters(record_batch)); - std::unique_ptr flight_info; - ASSERT_OK(prepared_statement->Execute(&flight_info)); + ASSERT_OK_AND_ASSIGN(auto flight_info, prepared_statement->Execute()); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -424,12 +418,11 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) ASSERT_TRUE(expected_table->Equals(*table)); } -Status ExecuteCountQuery(const std::string& query, int64_t* result) { - std::unique_ptr flight_info; - ARROW_RETURN_NOT_OK(sql_client->Execute({}, query, &flight_info)); +arrow::Result ExecuteCountQuery(const std::string& query) { + ARROW_ASSIGN_OR_RAISE(auto flight_info, sql_client->Execute({}, query)); - std::unique_ptr stream; - ARROW_RETURN_NOT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + 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)); @@ -437,19 +430,16 @@ Status ExecuteCountQuery(const std::string& query, int64_t* result) { const std::shared_ptr& result_array = table->column(0)->chunk(0); ARROW_ASSIGN_OR_RAISE(auto count_scalar, result_array->GetScalar(0)); - *result = reinterpret_cast(*count_scalar).value; - - return Status::OK(); + return reinterpret_cast(*count_scalar).value; } TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding) { - std::shared_ptr prepared_statement; - ASSERT_OK(sql_client->Prepare( - {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', ?)", - &prepared_statement)); + ASSERT_OK_AND_ASSIGN( + auto prepared_statement, + sql_client->Prepare( + {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', ?)")); - std::shared_ptr parameter_schema; - ASSERT_OK(prepared_statement->GetParameterSchema(¶meter_schema)); + ASSERT_OK_AND_ASSIGN(auto parameter_schema, prepared_statement->GetParameterSchema()); const std::shared_ptr& expected_parameter_schema = arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); @@ -482,54 +472,56 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding ASSERT_OK(prepared_statement->SetParameters(record_batch)); int64_t result; - ASSERT_OK(ExecuteCountQuery("SELECT COUNT(*) FROM intTable", &result)); + ASSERT_OK_AND_ASSIGN(result, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); ASSERT_EQ(3, result); - ASSERT_OK(prepared_statement->ExecuteUpdate(&result)); + ASSERT_OK_AND_ASSIGN(result, prepared_statement->ExecuteUpdate()); ASSERT_EQ(1, result); - ASSERT_OK(ExecuteCountQuery("SELECT COUNT(*) FROM intTable", &result)); + ASSERT_OK_AND_ASSIGN(result, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); ASSERT_EQ(4, result); - ASSERT_OK(sql_client->ExecuteUpdate( - {}, "DELETE FROM intTable WHERE keyName = 'new_value'", &result)); + ASSERT_OK_AND_ASSIGN( + result, + sql_client->ExecuteUpdate({}, "DELETE FROM intTable WHERE keyName = 'new_value'")); ASSERT_EQ(1, result); - ASSERT_OK(ExecuteCountQuery("SELECT COUNT(*) FROM intTable", &result)); + ASSERT_OK_AND_ASSIGN(result, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); ASSERT_EQ(3, result); } TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { - std::shared_ptr prepared_statement; - ASSERT_OK(sql_client->Prepare( - {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', 999)", - &prepared_statement)); + ASSERT_OK_AND_ASSIGN( + auto prepared_statement, + sql_client->Prepare( + {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', 999)")); int64_t result; - ASSERT_OK(ExecuteCountQuery("SELECT COUNT(*) FROM intTable", &result)); + ASSERT_OK_AND_ASSIGN(result, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); ASSERT_EQ(3, result); - ASSERT_OK(prepared_statement->ExecuteUpdate(&result)); + ASSERT_OK_AND_ASSIGN(result, prepared_statement->ExecuteUpdate()); ASSERT_EQ(1, result); - ASSERT_OK(ExecuteCountQuery("SELECT COUNT(*) FROM intTable", &result)); + ASSERT_OK_AND_ASSIGN(result, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); ASSERT_EQ(4, result); - ASSERT_OK(sql_client->ExecuteUpdate( - {}, "DELETE FROM intTable WHERE keyName = 'new_value'", &result)); + ASSERT_OK_AND_ASSIGN( + result, + sql_client->ExecuteUpdate({}, "DELETE FROM intTable WHERE keyName = 'new_value'")); ASSERT_EQ(1, result); - ASSERT_OK(ExecuteCountQuery("SELECT COUNT(*) FROM intTable", &result)); + ASSERT_OK_AND_ASSIGN(result, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); ASSERT_EQ(3, result); } TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { - std::unique_ptr flight_info; std::vector table_types; - ASSERT_OK(sql_client->GetPrimaryKeys({}, nullptr, nullptr, "int%", &flight_info)); + ASSERT_OK_AND_ASSIGN(auto flight_info, + sql_client->GetPrimaryKeys({}, nullptr, nullptr, "int%")); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -549,11 +541,11 @@ TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { } TEST(TestFlightSqlServer, TestCommandGetImportedKeys) { - std::unique_ptr flight_info; - ASSERT_OK(sql_client->GetImportedKeys({}, NULLPTR, NULLPTR, "intTable", &flight_info)); + ASSERT_OK_AND_ASSIGN(auto flight_info, + sql_client->GetImportedKeys({}, NULLPTR, NULLPTR, "intTable")); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -581,12 +573,11 @@ TEST(TestFlightSqlServer, TestCommandGetImportedKeys) { } TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { - std::unique_ptr flight_info; - ASSERT_OK( - sql_client->GetExportedKeys({}, NULLPTR, NULLPTR, "foreignTable", &flight_info)); + ASSERT_OK_AND_ASSIGN(auto flight_info, + sql_client->GetExportedKeys({}, NULLPTR, NULLPTR, "foreignTable")); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); @@ -614,12 +605,12 @@ TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { } TEST(TestFlightSqlServer, TestCommandGetCrossReference) { - std::unique_ptr flight_info; - ASSERT_OK(sql_client->GetCrossReference({}, NULLPTR, NULLPTR, "foreignTable", NULLPTR, - NULLPTR, "intTable", &flight_info)); + ASSERT_OK_AND_ASSIGN(auto flight_info, + sql_client->GetCrossReference({}, NULLPTR, NULLPTR, "foreignTable", + NULLPTR, NULLPTR, "intTable")); - std::unique_ptr stream; - ASSERT_OK(sql_client->DoGet({}, flight_info->endpoints()[0].ticket, &stream)); + ASSERT_OK_AND_ASSIGN(auto stream, + sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); diff --git a/cpp/src/arrow/flight/flight-sql/test_app_cli.cc b/cpp/src/arrow/flight/flight-sql/test_app_cli.cc index a83e696c627..eb035e6fc21 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app_cli.cc @@ -57,8 +57,7 @@ DEFINE_string(table, "", "Table"); Status PrintResultsForEndpoint(const FlightSqlClient& client, const FlightCallOptions& call_options, const FlightEndpoint& endpoint) { - std::unique_ptr stream; - ARROW_RETURN_NOT_OK(client.DoGet(call_options, endpoint.ticket, &stream)); + ARROW_ASSIGN_OR_RAISE(auto stream, client.DoGet(call_options, endpoint.ticket)); const arrow::Result>& schema = stream->GetSchema(); ARROW_RETURN_NOT_OK(schema); @@ -117,8 +116,8 @@ Status RunMain() { FlightSqlClient sqlClient(client); if (fLS::FLAGS_command == "ExecuteUpdate") { - int64_t rows; - ARROW_RETURN_NOT_OK(sqlClient.ExecuteUpdate(call_options, fLS::FLAGS_query, &rows)); + ARROW_ASSIGN_OR_RAISE(auto rows, + sqlClient.ExecuteUpdate(call_options, fLS::FLAGS_query)); std::cout << "Result: " << rows << std::endl; @@ -128,23 +127,20 @@ Status RunMain() { std::unique_ptr info; if (fLS::FLAGS_command == "Execute") { - ARROW_RETURN_NOT_OK(sqlClient.Execute(call_options, fLS::FLAGS_query, &info)); + ARROW_ASSIGN_OR_RAISE(info, sqlClient.Execute(call_options, fLS::FLAGS_query)); } else if (fLS::FLAGS_command == "GetCatalogs") { - ARROW_RETURN_NOT_OK(sqlClient.GetCatalogs(call_options, &info)); + ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetCatalogs(call_options)); } else if (fLS::FLAGS_command == "PreparedStatementExecute") { - std::shared_ptr prepared_statement; - - ARROW_RETURN_NOT_OK( - sqlClient.Prepare(call_options, fLS::FLAGS_query, &prepared_statement)); - ARROW_RETURN_NOT_OK(prepared_statement->Execute(&info)); - ARROW_RETURN_NOT_OK(PrintResults(sqlClient, call_options, info)); + ARROW_ASSIGN_OR_RAISE(auto prepared_statement, + sqlClient.Prepare(call_options, fLS::FLAGS_query)); + ARROW_ASSIGN_OR_RAISE(info, prepared_statement->Execute()); } else if (fLS::FLAGS_command == "PreparedStatementExecuteParameterBinding") { - std::shared_ptr prepared_statement; - ARROW_RETURN_NOT_OK(sqlClient.Prepare({}, fLS::FLAGS_query, &prepared_statement)); - std::shared_ptr parameter_schema; - std::shared_ptr result_set_schema; - ARROW_RETURN_NOT_OK(prepared_statement->GetParameterSchema(¶meter_schema)); - ARROW_RETURN_NOT_OK(prepared_statement->GetResultSetSchema(&result_set_schema)); + ARROW_ASSIGN_OR_RAISE(auto prepared_statement, + sqlClient.Prepare({}, fLS::FLAGS_query)); + ARROW_ASSIGN_OR_RAISE(auto parameter_schema, + prepared_statement->GetParameterSchema()); + ARROW_ASSIGN_OR_RAISE(auto result_set_schema, + prepared_statement->GetResultSetSchema()); std::cout << result_set_schema->ToString(false) << std::endl; arrow::Int64Builder int_builder; @@ -155,29 +151,31 @@ Status RunMain() { result = arrow::RecordBatch::Make(parameter_schema, 1, {int_array}); ARROW_RETURN_NOT_OK(prepared_statement->SetParameters(result)); - ARROW_RETURN_NOT_OK(prepared_statement->Execute(&info)); - ARROW_RETURN_NOT_OK(PrintResults(sqlClient, call_options, info)); + ARROW_ASSIGN_OR_RAISE(info, prepared_statement->Execute()); } else if (fLS::FLAGS_command == "GetSchemas") { - ARROW_RETURN_NOT_OK(sqlClient.GetSchemas(call_options, &fLS::FLAGS_catalog, - &fLS::FLAGS_schema, &info)); + ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetSchemas(call_options, &fLS::FLAGS_catalog, + &fLS::FLAGS_schema)); } else if (fLS::FLAGS_command == "GetTableTypes") { - ARROW_RETURN_NOT_OK(sqlClient.GetTableTypes(call_options, &info)); + ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetTableTypes(call_options)); } else if (fLS::FLAGS_command == "GetTables") { std::vector table_types = {}; bool include_schema = false; - ARROW_RETURN_NOT_OK(sqlClient.GetTables(call_options, &fLS::FLAGS_catalog, - &fLS::FLAGS_schema, &fLS::FLAGS_table, - include_schema, table_types, &info)); + ARROW_ASSIGN_OR_RAISE( + info, sqlClient.GetTables(call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, + &fLS::FLAGS_table, include_schema, table_types)); } else if (fLS::FLAGS_command == "GetExportedKeys") { - ARROW_RETURN_NOT_OK(sqlClient.GetExportedKeys( - call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, fLS::FLAGS_table, &info)); + ARROW_ASSIGN_OR_RAISE( + info, sqlClient.GetExportedKeys(call_options, &fLS::FLAGS_catalog, + &fLS::FLAGS_schema, fLS::FLAGS_table)); } else if (fLS::FLAGS_command == "GetImportedKeys") { - ARROW_RETURN_NOT_OK(sqlClient.GetImportedKeys( - call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, fLS::FLAGS_table, &info)); + ARROW_ASSIGN_OR_RAISE( + info, sqlClient.GetImportedKeys(call_options, &fLS::FLAGS_catalog, + &fLS::FLAGS_schema, fLS::FLAGS_table)); } else if (fLS::FLAGS_command == "GetPrimaryKeys") { - ARROW_RETURN_NOT_OK(sqlClient.GetPrimaryKeys( - call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, fLS::FLAGS_table, &info)); + ARROW_ASSIGN_OR_RAISE(info, + sqlClient.GetPrimaryKeys(call_options, &fLS::FLAGS_catalog, + &fLS::FLAGS_schema, fLS::FLAGS_table)); } if (info != NULLPTR && From 9a96bbdcd1befd5c93aa9bbd47dcd4a78c070666 Mon Sep 17 00:00:00 2001 From: Jose Almeida <53087160+jcralmeida@users.noreply.github.com> Date: Mon, 25 Oct 2021 14:23:50 -0300 Subject: [PATCH 157/237] [CPP] Change the way arrays are created in flight-sql tests (#179) * Create arrays using methods MakeArrayOfNull and ArrayFromJSON * Fix checkstyle * Remove macro declare binary array --- .../arrow/flight/flight-sql/server_test.cc | 264 ++++++++++-------- 1 file changed, 140 insertions(+), 124 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/server_test.cc b/cpp/src/arrow/flight/flight-sql/server_test.cc index b557e60e0c2..f86b07cfa86 100644 --- a/cpp/src/arrow/flight/flight-sql/server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/server_test.cc @@ -24,42 +24,8 @@ #include #include #include -#include #include -#define unparen(...) __VA_ARGS__ -#define DECLARE_ARRAY(ARRAY_NAME, TYPE_CLASS, DATA) \ - std::shared_ptr ARRAY_NAME; \ - { \ - arrow::TYPE_CLASS##Builder builder; \ - auto data = unparen DATA; \ - for (const auto& item : data) { \ - ASSERT_OK(builder.Append(item)); \ - } \ - ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ - } - -#define DECLARE_BINARY_ARRAY(ARRAY_NAME, DATA, LENGTH) \ - std::shared_ptr ARRAY_NAME; \ - { \ - arrow::Binary##Builder builder; \ - auto data = unparen DATA; \ - for (const auto& item : data) { \ - ASSERT_OK(builder.Append(item, LENGTH)); \ - } \ - ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ - } - -#define DECLARE_NULL_ARRAY(ARRAY_NAME, TYPE_CLASS, LENGTH) \ - std::shared_ptr ARRAY_NAME; \ - { \ - arrow::TYPE_CLASS##Builder builder; \ - for (int i = 0; i < LENGTH; i++) { \ - ASSERT_OK(builder.AppendNull()); \ - } \ - ASSERT_OK(builder.Finish(&(ARRAY_NAME))); \ - } - using ::testing::_; using ::testing::Ref; @@ -114,10 +80,15 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { arrow::schema({arrow::field("id", int64()), arrow::field("keyName", utf8()), arrow::field("value", int64()), arrow::field("foreignId", int64())}); - DECLARE_ARRAY(id_array, Int64, ({1, 2, 3})); - DECLARE_ARRAY(keyname_array, String, ({"one", "zero", "negative one"})); - DECLARE_ARRAY(value_array, Int64, ({1, 0, -1})); - DECLARE_ARRAY(foreignId_array, Int64, ({1, 1, 1})); + std::shared_ptr id_array; + std::shared_ptr keyname_array; + std::shared_ptr value_array; + std::shared_ptr foreignId_array; + ArrayFromVector({1, 2, 3}, &id_array); + ArrayFromVector({"one", "zero", "negative one"}, + &keyname_array); + ArrayFromVector({1, 0, -1}, &value_array); + ArrayFromVector({1, 1, 1}, &foreignId_array); const std::shared_ptr
& expected_table = Table::Make( expected_schema, {id_array, keyname_array, value_array, foreignId_array}); @@ -137,15 +108,13 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_NULL_ARRAY(catalog_name, String, 3); - DECLARE_NULL_ARRAY(schema_name, String, 3); - DECLARE_ARRAY(table_name, String, - ({ - "foreignTable", - "intTable", - "sqlite_sequence", - })); - DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); + ASSERT_OK_AND_ASSIGN(auto catalog_name, MakeArrayOfNull(utf8(), 3)) + ASSERT_OK_AND_ASSIGN(auto schema_name, MakeArrayOfNull(utf8(), 3)) + std::shared_ptr table_name; + ArrayFromVector( + {"foreignTable", "intTable", "sqlite_sequence"}, &table_name); + std::shared_ptr table_type; + ArrayFromVector({"table", "table", "table"}, &table_type); const std::shared_ptr
& expected_table = Table::Make( SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); @@ -167,10 +136,12 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_NULL_ARRAY(catalog_name, String, 1); - DECLARE_NULL_ARRAY(schema_name, String, 1); - DECLARE_ARRAY(table_name, String, ({"intTable"})); - DECLARE_ARRAY(table_type, String, ({"table"})); + ASSERT_OK_AND_ASSIGN(auto catalog_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto schema_name, MakeArrayOfNull(utf8(), 1)) + std::shared_ptr table_name; + std::shared_ptr table_type; + ArrayFromVector({"intTable"}, &table_name); + ArrayFromVector({"table"}, &table_type); const std::shared_ptr
& expected_table = Table::Make( SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); @@ -209,15 +180,18 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_NULL_ARRAY(catalog_name, String, 3); - DECLARE_NULL_ARRAY(schema_name, String, 3); - DECLARE_ARRAY(table_name, String, - ({ - "foreignTable", - "intTable", - "sqlite_sequence", - })); - DECLARE_ARRAY(table_type, String, ({"table", "table", "table"})); + ASSERT_OK_AND_ASSIGN(auto catalog_name, MakeArrayOfNull(utf8(), 3)) + ASSERT_OK_AND_ASSIGN(auto schema_name, MakeArrayOfNull(utf8(), 3)) + std::shared_ptr table_name; + ArrayFromVector( + { + "foreignTable", + "intTable", + "sqlite_sequence", + }, + &table_name); + std::shared_ptr table_type; + ArrayFromVector({"table", "table", "table"}, &table_type); const std::shared_ptr
& expected_table = Table::Make( SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); @@ -239,10 +213,12 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_NULL_ARRAY(catalog_name, String, 1); - DECLARE_NULL_ARRAY(schema_name, String, 1); - DECLARE_ARRAY(table_name, String, ({"intTable"})); - DECLARE_ARRAY(table_type, String, ({"table"})); + ASSERT_OK_AND_ASSIGN(auto catalog_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto schema_name, MakeArrayOfNull(utf8(), 1)) + std::shared_ptr table_name; + std::shared_ptr table_type; + ArrayFromVector({"intTable"}, &table_name); + ArrayFromVector({"table"}, &table_type); const std::shared_ptr schema_table = arrow::schema({arrow::field("id", int64(), true, NULL), @@ -252,14 +228,17 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { const arrow::Result>& value = ipc::SerializeSchema(*schema_table); - value.ValueOrDie()->data(), value.ValueOrDie()->size(); + std::shared_ptr schema_buffer; + + ASSERT_OK_AND_ASSIGN(schema_buffer, value); + std::shared_ptr table_schema; - DECLARE_BINARY_ARRAY(table_schema, ({value.ValueOrDie()->data()}), - value.ValueOrDie()->size()); + 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}); + Table::Make(SqlSchema::GetTablesSchemaWithIncludedSchema(), + {catalog_name, schema_name, table_name, table_type, table_schema}); ASSERT_TRUE(expected_table->Equals(*table)); } @@ -303,7 +282,8 @@ TEST(TestFlightSqlServer, TestCommandGetTableTypes) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_ARRAY(table_type, String, ({"table"})); + std::shared_ptr table_type; + ArrayFromVector({"table"}, &table_type); const std::shared_ptr
& expected_table = Table::Make(SqlSchema::GetTableTypesSchema(), {table_type}); @@ -347,10 +327,15 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQuery) { arrow::schema({arrow::field("id", int64()), arrow::field("keyName", utf8()), arrow::field("value", int64()), arrow::field("foreignId", int64())}); - DECLARE_ARRAY(id_array, Int64, ({1, 2, 3})); - DECLARE_ARRAY(keyname_array, String, ({"one", "zero", "negative one"})); - DECLARE_ARRAY(value_array, Int64, ({1, 0, -1})); - DECLARE_ARRAY(foreignId_array, Int64, ({1, 1, 1})); + std::shared_ptr id_array; + std::shared_ptr keyname_array; + std::shared_ptr value_array; + std::shared_ptr foreignId_array; + ArrayFromVector({1, 2, 3}, &id_array); + ArrayFromVector({"one", "zero", "negative one"}, + &keyname_array); + ArrayFromVector({1, 0, -1}, &value_array); + ArrayFromVector({1, 1, 1}, &foreignId_array); const std::shared_ptr
& expected_table = Table::Make( expected_schema, {id_array, keyname_array, value_array, foreignId_array}); @@ -407,10 +392,14 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) arrow::schema({arrow::field("id", int64()), arrow::field("keyName", utf8()), arrow::field("value", int64()), arrow::field("foreignId", int64())}); - DECLARE_ARRAY(id_array, Int64, ({1, 3})); - DECLARE_ARRAY(keyname_array, String, ({"one", "negative one"})); - DECLARE_ARRAY(value_array, Int64, ({1, -1})); - DECLARE_ARRAY(foreignId_array, Int64, ({1, 1})); + std::shared_ptr id_array; + std::shared_ptr keyname_array; + std::shared_ptr value_array; + std::shared_ptr foreignId_array; + ArrayFromVector({1, 3}, &id_array); + ArrayFromVector({"one", "negative one"}, &keyname_array); + ArrayFromVector({1, -1}, &value_array); + ArrayFromVector({1, 1}, &foreignId_array); const std::shared_ptr
& expected_table = Table::Make( expected_schema, {id_array, keyname_array, value_array, foreignId_array}); @@ -526,12 +515,16 @@ TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_NULL_ARRAY(catalog_name, String, 1); - DECLARE_NULL_ARRAY(schema_name, String, 1); - DECLARE_ARRAY(table_name, String, ({"intTable"})); - DECLARE_ARRAY(column_name, String, ({"id"})); - DECLARE_ARRAY(key_sequence, Int64, ({1})); - DECLARE_NULL_ARRAY(key_name, String, 1); + ASSERT_OK_AND_ASSIGN(auto catalog_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto schema_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto key_name, MakeArrayOfNull(utf8(), 1)) + + std::shared_ptr table_name; + std::shared_ptr column_name; + std::shared_ptr key_sequence; + ArrayFromVector({"intTable"}, &table_name); + ArrayFromVector({"id"}, &column_name); + ArrayFromVector({1}, &key_sequence); const std::shared_ptr
& expected_table = Table::Make( SqlSchema::GetPrimaryKeysSchema(), @@ -550,19 +543,27 @@ TEST(TestFlightSqlServer, TestCommandGetImportedKeys) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_NULL_ARRAY(pk_catalog_name, String, 1); - DECLARE_NULL_ARRAY(pk_schema_name, String, 1); - DECLARE_ARRAY(pk_table_name, String, ({"foreignTable"})); - DECLARE_ARRAY(pk_column_name, String, ({"id"})); - DECLARE_NULL_ARRAY(fk_catalog_name, String, 1); - DECLARE_NULL_ARRAY(fk_schema_name, String, 1); - DECLARE_ARRAY(fk_table_name, String, ({"intTable"})); - DECLARE_ARRAY(fk_column_name, String, ({"foreignId"})); - DECLARE_ARRAY(key_sequence, Int32, ({0})); - DECLARE_NULL_ARRAY(fk_key_name, String, 1); - DECLARE_NULL_ARRAY(pk_key_name, String, 1); - DECLARE_ARRAY(update_rule, UInt8, ({3})); - DECLARE_ARRAY(delete_rule, UInt8, ({3})); + std::shared_ptr pk_table_name; + std::shared_ptr pk_column_name; + std::shared_ptr fk_table_name; + std::shared_ptr fk_column_name; + std::shared_ptr key_sequence; + std::shared_ptr update_rule; + std::shared_ptr delete_rule; + ArrayFromVector({"foreignTable"}, &pk_table_name); + ArrayFromVector({"id"}, &pk_column_name); + ArrayFromVector({"intTable"}, &fk_table_name); + ArrayFromVector({"foreignId"}, &fk_column_name); + ArrayFromVector({0}, &key_sequence); + ArrayFromVector({3}, &update_rule); + ArrayFromVector({3}, &delete_rule); + + ASSERT_OK_AND_ASSIGN(auto pk_catalog_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto pk_schema_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto fk_catalog_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto fk_schema_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto fk_key_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto pk_key_name, MakeArrayOfNull(utf8(), 1)) const std::shared_ptr
& expected_table = Table::Make(SqlSchema::GetImportedKeysSchema(), @@ -582,19 +583,27 @@ TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_NULL_ARRAY(pk_catalog_name, String, 1); - DECLARE_NULL_ARRAY(pk_schema_name, String, 1); - DECLARE_ARRAY(pk_table_name, String, ({"foreignTable"})); - DECLARE_ARRAY(pk_column_name, String, ({"id"})); - DECLARE_NULL_ARRAY(fk_catalog_name, String, 1); - DECLARE_NULL_ARRAY(fk_schema_name, String, 1); - DECLARE_ARRAY(fk_table_name, String, ({"intTable"})); - DECLARE_ARRAY(fk_column_name, String, ({"foreignId"})); - DECLARE_ARRAY(key_sequence, Int32, ({0})); - DECLARE_NULL_ARRAY(fk_key_name, String, 1); - DECLARE_NULL_ARRAY(pk_key_name, String, 1); - DECLARE_ARRAY(update_rule, UInt8, ({3})); - DECLARE_ARRAY(delete_rule, UInt8, ({3})); + std::shared_ptr pk_table_name; + std::shared_ptr pk_column_name; + std::shared_ptr fk_table_name; + std::shared_ptr fk_column_name; + std::shared_ptr key_sequence; + std::shared_ptr update_rule; + std::shared_ptr delete_rule; + ArrayFromVector({"foreignTable"}, &pk_table_name); + ArrayFromVector({"id"}, &pk_column_name); + ArrayFromVector({"intTable"}, &fk_table_name); + ArrayFromVector({"foreignId"}, &fk_column_name); + ArrayFromVector({0}, &key_sequence); + ArrayFromVector({3}, &update_rule); + ArrayFromVector({3}, &delete_rule); + + ASSERT_OK_AND_ASSIGN(auto pk_catalog_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto pk_schema_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto fk_catalog_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto fk_schema_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto fk_key_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto pk_key_name, MakeArrayOfNull(utf8(), 1)) const std::shared_ptr
& expected_table = Table::Make(SqlSchema::GetExportedKeysSchema(), @@ -615,20 +624,27 @@ TEST(TestFlightSqlServer, TestCommandGetCrossReference) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - DECLARE_NULL_ARRAY(pk_catalog_name, String, 1); - DECLARE_NULL_ARRAY(pk_schema_name, String, 1); - DECLARE_ARRAY(pk_table_name, String, ({"foreignTable"})); - DECLARE_ARRAY(pk_column_name, String, ({"id"})); - DECLARE_NULL_ARRAY(fk_catalog_name, String, 1); - DECLARE_NULL_ARRAY(fk_schema_name, String, 1); - DECLARE_ARRAY(fk_table_name, String, ({"intTable"})); - DECLARE_ARRAY(fk_column_name, String, ({"foreignId"})); - DECLARE_ARRAY(key_sequence, Int32, ({0})); - DECLARE_NULL_ARRAY(fk_key_name, String, 1); - DECLARE_NULL_ARRAY(pk_key_name, String, 1); - DECLARE_ARRAY(update_rule, UInt8, ({3})); - DECLARE_ARRAY(delete_rule, UInt8, ({3})); - + std::shared_ptr pk_table_name; + std::shared_ptr pk_column_name; + std::shared_ptr fk_table_name; + std::shared_ptr fk_column_name; + std::shared_ptr key_sequence; + std::shared_ptr update_rule; + std::shared_ptr delete_rule; + ArrayFromVector({"foreignTable"}, &pk_table_name); + ArrayFromVector({"id"}, &pk_column_name); + ArrayFromVector({"intTable"}, &fk_table_name); + ArrayFromVector({"foreignId"}, &fk_column_name); + ArrayFromVector({0}, &key_sequence); + ArrayFromVector({3}, &update_rule); + ArrayFromVector({3}, &delete_rule); + + ASSERT_OK_AND_ASSIGN(auto pk_catalog_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto pk_schema_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto fk_catalog_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto fk_schema_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto fk_key_name, MakeArrayOfNull(utf8(), 1)) + ASSERT_OK_AND_ASSIGN(auto pk_key_name, MakeArrayOfNull(utf8(), 1)) const std::shared_ptr
& expected_table = Table::Make(SqlSchema::GetCrossReferenceSchema(), {pk_catalog_name, pk_schema_name, pk_table_name, pk_column_name, From ba7332757b197a34c2b3619690419816f0c62820 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 25 Oct 2021 15:00:08 -0300 Subject: [PATCH 158/237] [C++] Improvements on CMakeLists.txt (#178) * Remove extra options on CMakeLists.txt * Remove GRPC-related instructions from CMakeLists.txt and guard SQLite requirement on ARROW_BUILD_TESTS * Remove redudant dependencies on CMakeLists.txt --- .../arrow/flight/flight-sql/CMakeLists.txt | 75 +++++++++---------- cpp/src/arrow/flight/flight-sql/client.h | 4 +- cpp/src/arrow/flight/flight-sql/client_impl.h | 4 +- 3 files changed, 39 insertions(+), 44 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 4426e90bd0c..7491896fe9d 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -25,26 +25,16 @@ 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} gRPC::grpc_cpp_plugin) +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} ARGS - COMMAND ${ARROW_PROTOBUF_PROTOC} "-I${FLIGHT_SQL_PROTO_PATH}" - "--grpc_out=${CMAKE_CURRENT_BINARY_DIR}" - "--plugin=protoc-gen-grpc=$" - "${FLIGHT_SQL_PROTO}") + DEPENDS ${PROTO_DEPENDS}) set_source_files_properties(${FLIGHT_SQL_GENERATED_PROTO_FILES} PROPERTIES GENERATED TRUE) -add_custom_target(flight_sql_grpc_gen ALL DEPENDS ${FLIGHT_SQL_GENERATED_PROTO_FILES}) - -# -Werror / /WX cause try_compile to fail because there seems to be no -# way to pass -isystem $GRPC_INCLUDE_DIR instead of -I$GRPC_INCLUDE_DIR -set(CMAKE_CXX_FLAGS_BACKUP "${CMAKE_CXX_FLAGS}") -string(REPLACE "/WX" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") -string(REPLACE "-Werror " " " CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") +add_custom_target(flight_sql_protobuf_gen ALL DEPENDS ${FLIGHT_SQL_GENERATED_PROTO_FILES}) set(ARROW_FLIGHT_SQL_SRCS server.cc) @@ -60,7 +50,7 @@ add_arrow_lib(arrow_flight_sql PRECOMPILED_HEADERS "$<$:arrow/flight/flight-sql/pch.h>" DEPENDENCIES - flight_sql_grpc_gen + flight_sql_protobuf_gen SHARED_LINK_FLAGS ${ARROW_VERSION_SCRIPT_FLAGS} # Defined in cpp/arrow/CMakeLists.txt SHARED_LINK_LIBS @@ -79,34 +69,39 @@ else() ${ARROW_TEST_LINK_LIBS}) endif() -add_arrow_test(flight_sql_test - SOURCES - client_test.cc - server_test.cc - "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" - STATIC_LINK_LIBS - ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} - GTest::gtest - GTest::gmock - LABELS - "arrow_flight_sql") - add_executable(flight_sql_test_app test_app_cli.cc ${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc) target_link_libraries(flight_sql_test_app PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES}) -find_package(SQLite3) -include_directories(${SQLite3_INCLUDE_DIRS}) - -add_executable(flight_sql_test_server - server.cc - test_server_cli.cc - example/sqlite_statement.cc - example/sqlite_statement_batch_reader.cc - example/sqlite_server.cc - example/sqlite_tables_schema_batch_reader.cpp - "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc") -target_link_libraries(flight_sql_test_server - PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES} - ${SQLite3_LIBRARIES}) +add_arrow_test(flight_sql_test + SOURCES + client_test.cc + server_test.cc + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" + STATIC_LINK_LIBS + ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} + LABELS + "arrow_flight_sql") + +# Build test server for unit tests +if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) + find_package(SQLite3) + include_directories(${SQLite3_INCLUDE_DIRS}) + + add_executable(flight_sql_test_server + server.cc + test_server_cli.cc + example/sqlite_statement.cc + example/sqlite_statement_batch_reader.cc + example/sqlite_server.cc + example/sqlite_tables_schema_batch_reader.cpp + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc") + target_link_libraries(flight_sql_test_server + PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES} + ${SQLite3_LIBRARIES}) + + if(ARROW_BUILD_TESTS) + add_dependencies(arrow-flight-sql-test flight_sql_test_server) + endif() +endif() diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index d834f9aa2f1..f69d9c561c2 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -34,11 +34,11 @@ namespace internal { template class ARROW_EXPORT PreparedStatementT { + T* client; + FlightCallOptions options; pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; std::shared_ptr parameter_binding; - FlightCallOptions options; bool is_closed; - T* client; public: /// \brief Constructor for the PreparedStatement class. diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index b8d32c33bee..ae76ee51379 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -45,8 +45,8 @@ PreparedStatementT::PreparedStatementT( pb::sql::ActionCreatePreparedStatementResult& prepared_statement_result_, const FlightCallOptions& options_) : client(client_), - prepared_statement_result(std::move(prepared_statement_result_)), - options(options_) { + options(options_), + prepared_statement_result(std::move(prepared_statement_result_)) { is_closed = false; } From 5010c1baac182dc7b318e67a6c91da3d4cf66f79 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 25 Oct 2021 15:42:35 -0300 Subject: [PATCH 159/237] Fix flaky tests --- .../example/sqlite_statement_batch_reader.cc | 19 ++++++++++--------- .../example/sqlite_statement_batch_reader.h | 4 ++-- .../arrow/flight/flight-sql/server_test.cc | 7 +++---- 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc index 7d30a2ec145..1620622475c 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc @@ -73,9 +73,10 @@ std::shared_ptr SqliteStatementBatchReader::schema() const { return sche SqliteStatementBatchReader::SqliteStatementBatchReader( std::shared_ptr statement, std::shared_ptr schema) - : statement_(std::move(statement)), schema_(std::move(schema)) { - rc = SQLITE_OK; -} + : statement_(std::move(statement)), + schema_(std::move(schema)), + rc_(SQLITE_OK), + already_executed_(false) {} Status SqliteStatementBatchReader::Create( const std::shared_ptr& statement_, @@ -113,14 +114,14 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr* out) { ARROW_RETURN_NOT_OK(MakeBuilder(default_memory_pool(), field_type, &builders[i])); } - if (!already_executed) { - ARROW_RETURN_NOT_OK(statement_->Reset(&rc)); - ARROW_RETURN_NOT_OK(statement_->Step(&rc)); - already_executed = true; + if (!already_executed_) { + ARROW_RETURN_NOT_OK(statement_->Reset(&rc_)); + ARROW_RETURN_NOT_OK(statement_->Step(&rc_)); + already_executed_ = true; } int rows = 0; - while (rows < MAX_BATCH_SIZE && rc == SQLITE_ROW) { + while (rows < MAX_BATCH_SIZE && rc_ == SQLITE_ROW) { rows++; for (int i = 0; i < num_fields; i++) { const std::shared_ptr& field = schema_->field(i); @@ -149,7 +150,7 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr* out) { } } - ARROW_RETURN_NOT_OK(statement_->Step(&rc)); + ARROW_RETURN_NOT_OK(statement_->Step(&rc_)); } if (rows > 0) { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h index d8f25025687..e86ac0725b6 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h @@ -52,8 +52,8 @@ class SqliteStatementBatchReader : public RecordBatchReader { private: std::shared_ptr statement_; std::shared_ptr schema_; - int rc; - bool already_executed; + int rc_; + bool already_executed_; SqliteStatementBatchReader(std::shared_ptr statement, std::shared_ptr schema); diff --git a/cpp/src/arrow/flight/flight-sql/server_test.cc b/cpp/src/arrow/flight/flight-sql/server_test.cc index f86b07cfa86..89145967eed 100644 --- a/cpp/src/arrow/flight/flight-sql/server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/server_test.cc @@ -233,12 +233,11 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { ASSERT_OK_AND_ASSIGN(schema_buffer, value); std::shared_ptr table_schema; - ArrayFromVector( - {schema_buffer->ToString()}, &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}); + Table::Make(SqlSchema::GetTablesSchemaWithIncludedSchema(), + {catalog_name, schema_name, table_name, table_type, table_schema}); ASSERT_TRUE(expected_table->Equals(*table)); } From fdbe5f1a6368d3232bc62fea0cd4663e61e67cf8 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 26 Oct 2021 11:19:41 -0300 Subject: [PATCH 160/237] [C++] Wrap Protobufs on server (#182) * WIP: Wrap CommandStatementQuery protobuf into a struct * Wrap all Protobuf for commands on server * Change Parse methods as anonymous functions --- .../flight-sql/example/sqlite_server.cc | 261 +++++------ .../flight/flight-sql/example/sqlite_server.h | 73 ++- cpp/src/arrow/flight/flight-sql/server.cc | 442 ++++++++++++------ cpp/src/arrow/flight/flight-sql/server.h | 251 ++++++---- 4 files changed, 622 insertions(+), 405 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc index 844b1e80538..a8d0c82369e 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc @@ -57,31 +57,29 @@ std::shared_ptr GetArrowType(const char* sqlite_type) { } } -std::string PrepareQueryForGetTables(const pb::sql::CommandGetTables& command) { +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.has_catalog()) { - table_query << " and catalog_name='" << command.catalog() << "'"; + if (command.has_catalog) { + table_query << " and catalog_name='" << command.catalog << "'"; } - if (command.has_schema_filter_pattern()) { - table_query << " and schema_name LIKE '" << command.schema_filter_pattern() << "'"; + if (command.has_schema_filter_pattern) { + table_query << " and schema_name LIKE '" << command.schema_filter_pattern << "'"; } - if (command.has_table_name_filter_pattern()) { - table_query << " and table_name LIKE '" << command.table_name_filter_pattern() << "'"; + if (command.has_table_name_filter_pattern) { + table_query << " and table_name LIKE '" << command.table_name_filter_pattern << "'"; } - if (!command.table_types().empty()) { - google::protobuf::RepeatedPtrField types = command.table_types(); - + if (!command.table_types.empty()) { table_query << " and table_type IN ("; - int size = types.size(); - for (int i = 0; i < size; i++) { - table_query << "'" << types.at(i) << "'"; + 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 << ","; } @@ -205,13 +203,8 @@ Status DoGetSQLiteQuery(sqlite3* db, const std::string& query, Status GetFlightInfoForCommand(const FlightDescriptor& descriptor, std::unique_ptr* info, - const google::protobuf::Message& command, const std::shared_ptr& schema) { - google::protobuf::Any ticketParsed; - ticketParsed.PackFrom(command); - - std::vector endpoints{ - FlightEndpoint{{ticketParsed.SerializeAsString()}, {}}}; + std::vector endpoints{FlightEndpoint{{descriptor.cmd}, {}}}; ARROW_ASSIGN_OR_RAISE(auto result, FlightInfo::Make(*schema, descriptor, endpoints, -1, -1)) @@ -220,10 +213,11 @@ Status GetFlightInfoForCommand(const FlightDescriptor& descriptor, return Status::OK(); } -Status SQLiteFlightSqlServer::GetFlightInfoStatement( - const pb::sql::CommandStatementQuery& command, const ServerCallContext& context, - const FlightDescriptor& descriptor, std::unique_ptr* info) { - const std::string& query = command.query(); +Status SQLiteFlightSqlServer::GetFlightInfoStatement(const StatementQuery& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { + const std::string& query = command.query; std::shared_ptr statement; ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, query, &statement)); @@ -231,13 +225,7 @@ Status SQLiteFlightSqlServer::GetFlightInfoStatement( std::shared_ptr schema; ARROW_RETURN_NOT_OK(statement->GetSchema(&schema)); - pb::sql::TicketStatementQuery ticket_statement_query; - ticket_statement_query.set_statement_handle(query); - - google::protobuf::Any ticket; - ticket.PackFrom(ticket_statement_query); - - const std::string& ticket_string = ticket.SerializeAsString(); + std::string ticket_string = CreateStatementQueryTicket(query); std::vector endpoints{FlightEndpoint{{ticket_string}, {}}}; ARROW_ASSIGN_OR_RAISE(auto result, FlightInfo::Make(*schema, descriptor, endpoints, -1, -1)) @@ -247,10 +235,10 @@ Status SQLiteFlightSqlServer::GetFlightInfoStatement( return Status::OK(); } -Status SQLiteFlightSqlServer::DoGetStatement(const pb::sql::TicketStatementQuery& command, +Status SQLiteFlightSqlServer::DoGetStatement(const StatementQueryTicket& command, const ServerCallContext& context, std::unique_ptr* result) { - const std::string& sql = command.statement_handle(); + const std::string& sql = command.statement_handle; std::shared_ptr statement; ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, sql, &statement)); @@ -266,9 +254,7 @@ Status SQLiteFlightSqlServer::DoGetStatement(const pb::sql::TicketStatementQuery Status SQLiteFlightSqlServer::GetFlightInfoCatalogs(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - pb::sql::CommandGetCatalogs command; - return GetFlightInfoForCommand(descriptor, info, command, - SqlSchema::GetCatalogsSchema()); + return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetCatalogsSchema()); } Status SQLiteFlightSqlServer::DoGetCatalogs(const ServerCallContext& context, @@ -288,14 +274,14 @@ Status SQLiteFlightSqlServer::DoGetCatalogs(const ServerCallContext& context, return Status::OK(); } -Status SQLiteFlightSqlServer::GetFlightInfoSchemas( - const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, - const FlightDescriptor& descriptor, std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, command, - SqlSchema::GetSchemasSchema()); +Status SQLiteFlightSqlServer::GetFlightInfoSchemas(const GetSchemas& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { + return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetSchemasSchema()); } -Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& command, +Status SQLiteFlightSqlServer::DoGetSchemas(const GetSchemas& command, const ServerCallContext& context, std::unique_ptr* result) { // As SQLite doesn't support schemas, this will return an empty record batch. @@ -315,17 +301,13 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const pb::sql::CommandGetSchemas& com return Status::OK(); } -Status SQLiteFlightSqlServer::GetFlightInfoTables( - const pb::sql::CommandGetTables& command, const ServerCallContext& context, - const FlightDescriptor& descriptor, std::unique_ptr* info) { - google::protobuf::Any ticketParsed; +Status SQLiteFlightSqlServer::GetFlightInfoTables(const GetTables& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { + std::vector endpoints{FlightEndpoint{{descriptor.cmd}, {}}}; - ticketParsed.PackFrom(command); - - std::vector endpoints{ - FlightEndpoint{{ticketParsed.SerializeAsString()}, {}}}; - - bool include_schema = command.include_schema(); + bool include_schema = command.include_schema; ARROW_ASSIGN_OR_RAISE( auto result, @@ -337,7 +319,7 @@ Status SQLiteFlightSqlServer::GetFlightInfoTables( return Status::OK(); } -Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& command, +Status SQLiteFlightSqlServer::DoGetTables(const GetTables& command, const ServerCallContext& context, std::unique_ptr* result) { std::string query = PrepareQueryForGetTables(command); @@ -349,7 +331,7 @@ Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& comma ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create( statement, SqlSchema::GetTablesSchema(), &reader)); - if (command.include_schema()) { + if (command.include_schema) { std::shared_ptr table_schema_reader = std::make_shared(reader, query, db_); *result = @@ -361,41 +343,32 @@ Status SQLiteFlightSqlServer::DoGetTables(const pb::sql::CommandGetTables& comma return Status::OK(); } -Status SQLiteFlightSqlServer::DoPutCommandStatementUpdate( - const pb::sql::CommandStatementUpdate& command, const ServerCallContext& context, - std::unique_ptr& reader, - std::unique_ptr& writer) { - const std::string& sql = command.query(); +arrow::Result SQLiteFlightSqlServer::DoPutCommandStatementUpdate( + const StatementUpdate& command, const ServerCallContext& context, + std::unique_ptr& reader) { + const std::string& sql = command.query; std::shared_ptr statement; ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, sql, &statement)); - pb::sql::DoPutUpdateResult result; - int64_t record_count; ARROW_RETURN_NOT_OK(statement->ExecuteUpdate(&record_count)); - result.set_record_count(record_count); - - const std::shared_ptr& buffer = Buffer::FromString(result.SerializeAsString()); - ARROW_RETURN_NOT_OK(writer->WriteMetadata(*buffer)); - - return Status::OK(); + return record_count; } -Status SQLiteFlightSqlServer::CreatePreparedStatement( - const pb::sql::ActionCreatePreparedStatementRequest& request, - const ServerCallContext& context, std::unique_ptr* result) { +arrow::Result +SQLiteFlightSqlServer::CreatePreparedStatement( + const ActionCreatePreparedStatementRequest& request, + const ServerCallContext& context) { std::shared_ptr statement; - ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, request.query(), &statement)); + ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, request.query, &statement)); boost::uuids::uuid uuid = uuid_generator_(); prepared_statements_[uuid] = statement; std::shared_ptr dataset_schema; ARROW_RETURN_NOT_OK(statement->GetSchema(&dataset_schema)); - ARROW_ASSIGN_OR_RAISE(auto serialized_dataset_schema, - ipc::SerializeSchema(*dataset_schema)); sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); const int parameter_count = sqlite3_bind_parameter_count(stmt); @@ -419,27 +392,19 @@ Status SQLiteFlightSqlServer::CreatePreparedStatement( } const std::shared_ptr& parameter_schema = arrow::schema(parameter_fields); - ARROW_ASSIGN_OR_RAISE(auto serialized_parameter_schema, - ipc::SerializeSchema(*parameter_schema)); - - pb::sql::ActionCreatePreparedStatementResult action_result; - action_result.set_dataset_schema(serialized_dataset_schema->ToString()); - action_result.set_parameter_schema(serialized_parameter_schema->ToString()); - action_result.set_prepared_statement_handle(boost::uuids::to_string(uuid)); - google::protobuf::Any any; - any.PackFrom(action_result); + ActionCreatePreparedStatementResult result{ + .dataset_schema = dataset_schema, + .parameter_schema = parameter_schema, + .prepared_statement_handle = boost::uuids::to_string(uuid)}; - auto buf = Buffer::FromString(any.SerializeAsString()); - *result = std::unique_ptr(new SimpleResultStream({Result{buf}})); - - return Status::OK(); + return result; } Status SQLiteFlightSqlServer::ClosePreparedStatement( - const pb::sql::ActionClosePreparedStatementRequest& request, - const ServerCallContext& context, std::unique_ptr* result) { - const std::string& prepared_statement_handle = request.prepared_statement_handle(); + const ActionClosePreparedStatementRequest& request, const ServerCallContext& context, + std::unique_ptr* result) { + const std::string& prepared_statement_handle = request.prepared_statement_handle; const auto& uuid = boost::lexical_cast(prepared_statement_handle); auto search = prepared_statements_.find(uuid); @@ -455,10 +420,9 @@ Status SQLiteFlightSqlServer::ClosePreparedStatement( } Status SQLiteFlightSqlServer::GetFlightInfoPreparedStatement( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info) { - const std::string& prepared_statement_handle = command.prepared_statement_handle(); + const PreparedStatementQuery& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { + const std::string& prepared_statement_handle = command.prepared_statement_handle; const auto& uuid = boost::lexical_cast(prepared_statement_handle); auto search = prepared_statements_.find(uuid); @@ -471,13 +435,13 @@ Status SQLiteFlightSqlServer::GetFlightInfoPreparedStatement( std::shared_ptr schema; ARROW_RETURN_NOT_OK(statement->GetSchema(&schema)); - return GetFlightInfoForCommand(descriptor, info, command, schema); + return GetFlightInfoForCommand(descriptor, info, schema); } Status SQLiteFlightSqlServer::DoGetPreparedStatement( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, std::unique_ptr* result) { - const std::string& prepared_statement_handle = command.prepared_statement_handle(); + const PreparedStatementQuery& command, const ServerCallContext& context, + std::unique_ptr* result) { + const std::string& prepared_statement_handle = command.prepared_statement_handle; const auto& uuid = boost::lexical_cast(prepared_statement_handle); auto search = prepared_statements_.find(uuid); @@ -510,10 +474,10 @@ Status SQLiteFlightSqlServer::GetStatementByHandle( } Status SQLiteFlightSqlServer::DoPutPreparedStatementQuery( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, std::unique_ptr& reader, + const PreparedStatementQuery& command, const ServerCallContext& context, + std::unique_ptr& reader, std::unique_ptr& writer) { - const std::string& prepared_statement_handle = command.prepared_statement_handle(); + const std::string& prepared_statement_handle = command.prepared_statement_handle; std::shared_ptr statement; ARROW_RETURN_NOT_OK(GetStatementByHandle(prepared_statement_handle, &statement)); @@ -523,11 +487,10 @@ Status SQLiteFlightSqlServer::DoPutPreparedStatementQuery( return Status::OK(); } -Status SQLiteFlightSqlServer::DoPutPreparedStatementUpdate( - const pb::sql::CommandPreparedStatementUpdate& command, - const ServerCallContext& context, std::unique_ptr& reader, - std::unique_ptr& writer) { - const std::string& prepared_statement_handle = command.prepared_statement_handle(); +arrow::Result SQLiteFlightSqlServer::DoPutPreparedStatementUpdate( + const PreparedStatementUpdate& command, const ServerCallContext& context, + std::unique_ptr& reader) { + const std::string& prepared_statement_handle = command.prepared_statement_handle; std::shared_ptr statement; ARROW_RETURN_NOT_OK(GetStatementByHandle(prepared_statement_handle, &statement)); @@ -538,21 +501,13 @@ Status SQLiteFlightSqlServer::DoPutPreparedStatementUpdate( int64_t record_count; ARROW_RETURN_NOT_OK(statement->ExecuteUpdate(&record_count)); - pb::sql::DoPutUpdateResult result; - result.set_record_count(record_count); - - const std::shared_ptr& buffer = Buffer::FromString(result.SerializeAsString()); - ARROW_RETURN_NOT_OK(writer->WriteMetadata(*buffer)); - - return Status::OK(); + return record_count; } Status SQLiteFlightSqlServer::GetFlightInfoTableTypes(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - pb::sql::CommandGetTableTypes command; - return GetFlightInfoForCommand(descriptor, info, command, - SqlSchema::GetTableTypesSchema()); + return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetTableTypesSchema()); } Status SQLiteFlightSqlServer::DoGetTableTypes(const ServerCallContext& context, @@ -563,14 +518,13 @@ Status SQLiteFlightSqlServer::DoGetTableTypes(const ServerCallContext& context, } Status SQLiteFlightSqlServer::GetFlightInfoPrimaryKeys( - const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, + const GetPrimaryKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, command, - SqlSchema::GetPrimaryKeysSchema()); + return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetPrimaryKeysSchema()); } Status SQLiteFlightSqlServer::DoGetPrimaryKeys( - const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, + const GetPrimaryKeys& command, const ServerCallContext& context, std::unique_ptr* result) { std::stringstream table_query; @@ -583,15 +537,15 @@ Status SQLiteFlightSqlServer::DoGetPrimaryKeys( "table_name, type as table_type\n" "FROM sqlite_master) where 1=1 and pk != 0"; - if (command.has_catalog()) { - table_query << " and catalog_name LIKE '" << command.catalog() << "'"; + if (command.has_catalog) { + table_query << " and catalog_name LIKE '" << command.catalog << "'"; } - if (command.has_schema()) { - table_query << " and schema_name LIKE '" << command.schema() << "'"; + if (command.has_schema) { + table_query << " and schema_name LIKE '" << command.schema << "'"; } - table_query << " and table_name LIKE '" << command.table() << "'"; + table_query << " and table_name LIKE '" << command.table << "'"; return DoGetSQLiteQuery(db_, table_query.str(), SqlSchema::GetPrimaryKeysSchema(), result); @@ -631,21 +585,20 @@ std::string PrepareQueryForGetImportedOrExportedKeys(const std::string& filter) } Status SQLiteFlightSqlServer::GetFlightInfoImportedKeys( - const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, + const GetImportedKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, command, - SqlSchema::GetImportedKeysSchema()); + return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetImportedKeysSchema()); } Status SQLiteFlightSqlServer::DoGetImportedKeys( - const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, + const GetImportedKeys& command, const ServerCallContext& context, std::unique_ptr* result) { - std::string filter = "fk_table_name = '" + command.table() + "'"; - if (command.has_catalog()) { - filter += " AND fk_catalog_name = '" + command.catalog() + "'"; + std::string filter = "fk_table_name = '" + command.table + "'"; + if (command.has_catalog) { + filter += " AND fk_catalog_name = '" + command.catalog + "'"; } - if (command.has_schema()) { - filter += " AND fk_schema_name = '" + command.schema() + "'"; + if (command.has_schema) { + filter += " AND fk_schema_name = '" + command.schema + "'"; } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); @@ -653,21 +606,20 @@ Status SQLiteFlightSqlServer::DoGetImportedKeys( } Status SQLiteFlightSqlServer::GetFlightInfoExportedKeys( - const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, + const GetExportedKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, command, - SqlSchema::GetExportedKeysSchema()); + return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetExportedKeysSchema()); } Status SQLiteFlightSqlServer::DoGetExportedKeys( - const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, + const GetExportedKeys& command, const ServerCallContext& context, std::unique_ptr* result) { - std::string filter = "pk_table_name = '" + command.table() + "'"; - if (command.has_catalog()) { - filter += " AND pk_catalog_name = '" + command.catalog() + "'"; + std::string filter = "pk_table_name = '" + command.table + "'"; + if (command.has_catalog) { + filter += " AND pk_catalog_name = '" + command.catalog + "'"; } - if (command.has_schema()) { - filter += " AND pk_schema_name = '" + command.schema() + "'"; + if (command.has_schema) { + filter += " AND pk_schema_name = '" + command.schema + "'"; } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); @@ -675,29 +627,28 @@ Status SQLiteFlightSqlServer::DoGetExportedKeys( } Status SQLiteFlightSqlServer::GetFlightInfoCrossReference( - const pb::sql::CommandGetCrossReference& command, const ServerCallContext& context, + const GetCrossReference& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, command, - SqlSchema::GetCrossReferenceSchema()); + return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetCrossReferenceSchema()); } Status SQLiteFlightSqlServer::DoGetCrossReference( - const pb::sql::CommandGetCrossReference& command, const ServerCallContext& context, + const GetCrossReference& command, const ServerCallContext& context, std::unique_ptr* result) { - std::string filter = "pk_table_name = '" + command.pk_table() + "'"; - if (command.has_pk_catalog()) { - filter += " AND pk_catalog_name = '" + command.pk_catalog() + "'"; + std::string filter = "pk_table_name = '" + command.pk_table + "'"; + if (command.has_pk_catalog) { + filter += " AND pk_catalog_name = '" + command.pk_catalog + "'"; } - if (command.has_pk_schema()) { - filter += " AND pk_schema_name = '" + command.pk_schema() + "'"; + if (command.has_pk_schema) { + filter += " AND pk_schema_name = '" + command.pk_schema + "'"; } - filter += " AND fk_table_name = '" + command.fk_table() + "'"; - if (command.has_fk_catalog()) { - filter += " AND fk_catalog_name = '" + command.fk_catalog() + "'"; + filter += " AND fk_table_name = '" + command.fk_table + "'"; + if (command.has_fk_catalog) { + filter += " AND fk_catalog_name = '" + command.fk_catalog + "'"; } - if (command.has_fk_schema()) { - filter += " AND fk_schema_name = '" + command.fk_schema() + "'"; + if (command.has_fk_schema) { + filter += " AND fk_schema_name = '" + command.fk_schema + "'"; } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index d3890c6dba7..9b5bb0b5d02 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -60,12 +60,12 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { /// SQLite database. void ExecuteSql(const std::string& sql); - Status GetFlightInfoStatement(const pb::sql::CommandStatementQuery& command, + Status GetFlightInfoStatement(const StatementQuery& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetStatement(const pb::sql::TicketStatementQuery& command, + Status DoGetStatement(const StatementQueryTicket& command, const ServerCallContext& context, std::unique_ptr* result) override; Status GetFlightInfoCatalogs(const ServerCallContext& context, @@ -73,81 +73,74 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { std::unique_ptr* info) override; Status DoGetCatalogs(const ServerCallContext& context, std::unique_ptr* result) override; - Status GetFlightInfoSchemas(const pb::sql::CommandGetSchemas& command, - const ServerCallContext& context, + Status GetFlightInfoSchemas(const GetSchemas& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetSchemas(const pb::sql::CommandGetSchemas& command, - const ServerCallContext& context, + Status DoGetSchemas(const GetSchemas& command, const ServerCallContext& context, std::unique_ptr* result) override; - Status DoPutCommandStatementUpdate( - const pb::sql::CommandStatementUpdate& update, const ServerCallContext& context, - std::unique_ptr& reader, - std::unique_ptr& writer) override; - Status CreatePreparedStatement( - const pb::sql::ActionCreatePreparedStatementRequest& request, - const ServerCallContext& context, std::unique_ptr* result) override; - Status ClosePreparedStatement( - const pb::sql::ActionClosePreparedStatementRequest& request, - const ServerCallContext& context, std::unique_ptr* result) override; - Status GetFlightInfoPreparedStatement( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info) override; - Status DoGetPreparedStatement(const pb::sql::CommandPreparedStatementQuery& command, + arrow::Result DoPutCommandStatementUpdate( + const StatementUpdate& update, const ServerCallContext& context, + std::unique_ptr& reader) override; + arrow::Result CreatePreparedStatement( + const ActionCreatePreparedStatementRequest& request, + const ServerCallContext& context) override; + Status ClosePreparedStatement(const ActionClosePreparedStatementRequest& request, + const ServerCallContext& context, + std::unique_ptr* result) override; + Status GetFlightInfoPreparedStatement(const PreparedStatementQuery& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) override; + Status DoGetPreparedStatement(const PreparedStatementQuery& command, const ServerCallContext& context, std::unique_ptr* result) override; Status DoPutPreparedStatementQuery( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, std::unique_ptr& reader, - std::unique_ptr& writer) override; - Status DoPutPreparedStatementUpdate( - const pb::sql::CommandPreparedStatementUpdate& command, - const ServerCallContext& context, std::unique_ptr& reader, + const PreparedStatementQuery& command, const ServerCallContext& context, + std::unique_ptr& reader, std::unique_ptr& writer) override; + arrow::Result DoPutPreparedStatementUpdate( + const PreparedStatementUpdate& command, const ServerCallContext& context, + std::unique_ptr& reader) override; - Status GetFlightInfoTables(const pb::sql::CommandGetTables& command, - const ServerCallContext& context, + Status GetFlightInfoTables(const GetTables& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetTables(const pb::sql::CommandGetTables& command, - const ServerCallContext& context, + Status DoGetTables(const GetTables& command, const ServerCallContext& context, std::unique_ptr* result) override; Status GetFlightInfoTableTypes(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) override; Status DoGetTableTypes(const ServerCallContext& context, std::unique_ptr* result) override; - Status GetFlightInfoImportedKeys(const pb::sql::CommandGetImportedKeys& command, + Status GetFlightInfoImportedKeys(const GetImportedKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetImportedKeys(const pb::sql::CommandGetImportedKeys& command, + Status DoGetImportedKeys(const GetImportedKeys& command, const ServerCallContext& context, std::unique_ptr* result) override; - Status GetFlightInfoExportedKeys(const pb::sql::CommandGetExportedKeys& command, + Status GetFlightInfoExportedKeys(const GetExportedKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetExportedKeys(const pb::sql::CommandGetExportedKeys& command, + Status DoGetExportedKeys(const GetExportedKeys& command, const ServerCallContext& context, std::unique_ptr* result) override; - Status GetFlightInfoCrossReference(const pb::sql::CommandGetCrossReference& command, + Status GetFlightInfoCrossReference(const GetCrossReference& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetCrossReference(const pb::sql::CommandGetCrossReference& command, + Status DoGetCrossReference(const GetCrossReference& command, const ServerCallContext& context, std::unique_ptr* result) override; - Status GetFlightInfoPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, + Status GetFlightInfoPrimaryKeys(const GetPrimaryKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, - const ServerCallContext& context, + Status DoGetPrimaryKeys(const GetPrimaryKeys& command, const ServerCallContext& context, std::unique_ptr* result) override; private: diff --git a/cpp/src/arrow/flight/flight-sql/server.cc b/cpp/src/arrow/flight/flight-sql/server.cc index 17b2ec902da..b8682e947d1 100644 --- a/cpp/src/arrow/flight/flight-sql/server.cc +++ b/cpp/src/arrow/flight/flight-sql/server.cc @@ -18,13 +18,177 @@ // Interfaces to use for defining Flight RPC servers. API should be considered // experimental for now -#include "arrow/flight/flight-sql/server.h" +#include "server.h" +#include +#include +#include + +#include "arrow/api.h" #include "arrow/buffer.h" +std::string arrow::flight::sql::FlightSqlServerBase::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); + + const std::string& ticket_string = ticket.SerializeAsString(); + return ticket_string; +} + namespace arrow { namespace flight { namespace sql { + +namespace pb = arrow::flight::protocol; + +GetCrossReference ParseCommandGetCrossReference(const google::protobuf::Any& any) { + pb::sql::CommandGetCrossReference command; + any.UnpackTo(&command); + + return (GetCrossReference){.has_pk_catalog = command.has_pk_catalog(), + .pk_catalog = command.pk_catalog(), + .has_pk_schema = command.has_pk_schema(), + .pk_schema = command.pk_schema(), + .pk_table = command.pk_table(), + .has_fk_catalog = command.has_fk_catalog(), + .fk_catalog = command.fk_catalog(), + .has_fk_schema = command.has_fk_schema(), + .fk_schema = command.fk_schema(), + .fk_table = command.fk_table()}; +} + +GetImportedKeys ParseCommandGetImportedKeys(const google::protobuf::Any& any) { + pb::sql::CommandGetImportedKeys command; + any.UnpackTo(&command); + + return (GetImportedKeys){.has_catalog = command.has_catalog(), + .catalog = command.catalog(), + .has_schema = command.has_schema(), + .schema = command.schema(), + .table = command.table()}; +} + +GetExportedKeys ParseCommandGetExportedKeys(const google::protobuf::Any& any) { + pb::sql::CommandGetExportedKeys command; + any.UnpackTo(&command); + + return (GetExportedKeys){.has_catalog = command.has_catalog(), + .catalog = command.catalog(), + .has_schema = command.has_schema(), + .schema = command.schema(), + .table = command.table()}; +} + +GetPrimaryKeys ParseCommandGetPrimaryKeys(const google::protobuf::Any& any) { + pb::sql::CommandGetPrimaryKeys command; + any.UnpackTo(&command); + + return (GetPrimaryKeys){.has_catalog = command.has_catalog(), + .catalog = command.catalog(), + .has_schema = command.has_schema(), + .schema = command.schema(), + .table = command.table()}; +} + +GetSqlInfo ParseCommandGetSqlInfo(const google::protobuf::Any& any) { + pb::sql::CommandGetSqlInfo command; + any.UnpackTo(&command); + + return (GetSqlInfo){}; +} + +GetSchemas ParseCommandGetSchemas(const google::protobuf::Any& any) { + pb::sql::CommandGetSchemas command; + any.UnpackTo(&command); + + return (GetSchemas){ + .has_catalog = command.has_catalog(), + .catalog = command.catalog(), + .has_schema_filter_pattern = command.has_schema_filter_pattern(), + .schema_filter_pattern = command.schema_filter_pattern(), + }; +} + +PreparedStatementQuery ParseCommandPreparedStatementQuery( + const google::protobuf::Any& any) { + pb::sql::CommandPreparedStatementQuery command; + any.UnpackTo(&command); + + return (PreparedStatementQuery){.prepared_statement_handle = + command.prepared_statement_handle()}; +} + +StatementQuery ParseCommandStatementQuery(const google::protobuf::Any& any) { + pb::sql::CommandStatementQuery command; + any.UnpackTo(&command); + + return (StatementQuery){.query = command.query()}; +} + +GetTables ParseCommandGetTables(const google::protobuf::Any& anyCommand) { + pb::sql::CommandGetTables command; + anyCommand.UnpackTo(&command); + + std::vector table_types; + table_types.reserve(command.table_types_size()); + for (const auto& item : command.table_types()) { + table_types.push_back(item); + } + return (GetTables){ + .has_catalog = command.has_catalog(), + .catalog = command.catalog(), + .has_schema_filter_pattern = command.has_schema_filter_pattern(), + .schema_filter_pattern = command.schema_filter_pattern(), + .has_table_name_filter_pattern = command.has_table_name_filter_pattern(), + .table_name_filter_pattern = command.table_name_filter_pattern(), + .table_types = table_types, + .include_schema = command.include_schema()}; +} + +StatementQueryTicket ParseStatementQueryTicket(const google::protobuf::Any& anyCommand) { + pb::sql::TicketStatementQuery command; + anyCommand.UnpackTo(&command); + + return (StatementQueryTicket){.statement_handle = command.statement_handle()}; +} + +StatementUpdate ParseCommandStatementUpdate(const google::protobuf::Any& any) { + pb::sql::CommandStatementUpdate command; + any.UnpackTo(&command); + + return (StatementUpdate){.query = command.query()}; +} + +PreparedStatementUpdate ParseCommandPreparedStatementUpdate( + const google::protobuf::Any& any) { + pb::sql::CommandPreparedStatementUpdate command; + any.UnpackTo(&command); + + return (PreparedStatementUpdate){.prepared_statement_handle = + command.prepared_statement_handle()}; +} + +ActionCreatePreparedStatementRequest ParseActionCreatePreparedStatementRequest( + const google::protobuf::Any& anyCommand) { + pb::sql::ActionCreatePreparedStatementRequest command; + anyCommand.UnpackTo(&command); + + return (ActionCreatePreparedStatementRequest){.query = command.query()}; +} + +ActionClosePreparedStatementRequest ParseActionClosePreparedStatementRequest( + const google::protobuf::Any& anyCommand) { + pb::sql::ActionClosePreparedStatementRequest command; + anyCommand.UnpackTo(&command); + + return (ActionClosePreparedStatementRequest){.prepared_statement_handle = + command.prepared_statement_handle()}; +} + Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, std::unique_ptr* info) { @@ -32,49 +196,36 @@ Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size())); if (any.Is()) { - pb::sql::CommandStatementQuery command; - any.UnpackTo(&command); - return GetFlightInfoStatement(command, context, request, info); + StatementQuery internal_command = ParseCommandStatementQuery(any); + return GetFlightInfoStatement(internal_command, context, request, info); } else if (any.Is()) { - pb::sql::CommandPreparedStatementQuery command; - any.UnpackTo(&command); - return GetFlightInfoPreparedStatement(command, context, request, info); + PreparedStatementQuery internal_command = ParseCommandPreparedStatementQuery(any); + return GetFlightInfoPreparedStatement(internal_command, context, request, info); } else if (any.Is()) { - pb::sql::CommandGetCatalogs command; - any.UnpackTo(&command); return GetFlightInfoCatalogs(context, request, info); } else if (any.Is()) { - pb::sql::CommandGetSchemas command; - any.UnpackTo(&command); - return GetFlightInfoSchemas(command, context, request, info); + GetSchemas internal_command = ParseCommandGetSchemas(any); + return GetFlightInfoSchemas(internal_command, context, request, info); } else if (any.Is()) { - pb::sql::CommandGetTables command; - any.UnpackTo(&command); + GetTables command = ParseCommandGetTables(any); return GetFlightInfoTables(command, context, request, info); } else if (any.Is()) { - pb::sql::CommandGetTableTypes command; - any.UnpackTo(&command); return GetFlightInfoTableTypes(context, request, info); } else if (any.Is()) { - pb::sql::CommandGetSqlInfo command; - any.UnpackTo(&command); - return GetFlightInfoSqlInfo(command, context, request, info); + GetSqlInfo internal_command = ParseCommandGetSqlInfo(any); + return GetFlightInfoSqlInfo(internal_command, context, request, info); } else if (any.Is()) { - pb::sql::CommandGetPrimaryKeys command; - any.UnpackTo(&command); - return GetFlightInfoPrimaryKeys(command, context, request, info); + GetPrimaryKeys internal_command = ParseCommandGetPrimaryKeys(any); + return GetFlightInfoPrimaryKeys(internal_command, context, request, info); } else if (any.Is()) { - pb::sql::CommandGetExportedKeys command; - any.UnpackTo(&command); - return GetFlightInfoExportedKeys(command, context, request, info); + GetExportedKeys internal_command = ParseCommandGetExportedKeys(any); + return GetFlightInfoExportedKeys(internal_command, context, request, info); } else if (any.Is()) { - pb::sql::CommandGetImportedKeys command; - any.UnpackTo(&command); - return GetFlightInfoImportedKeys(command, context, request, info); + GetImportedKeys internal_command = ParseCommandGetImportedKeys(any); + return GetFlightInfoImportedKeys(internal_command, context, request, info); } else if (any.Is()) { - pb::sql::CommandGetCrossReference command; - any.UnpackTo(&command); - return GetFlightInfoCrossReference(command, context, request, info); + GetCrossReference internal_command = ParseCommandGetCrossReference(any); + return GetFlightInfoCrossReference(internal_command, context, request, info); } return Status::Invalid("The defined request is invalid."); @@ -88,49 +239,37 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket static_cast(request.ticket.size())); if (anyCommand.Is()) { - pb::sql::TicketStatementQuery command; - anyCommand.UnpackTo(&command); + StatementQueryTicket command = ParseStatementQueryTicket(anyCommand); return DoGetStatement(command, context, stream); } else if (anyCommand.Is()) { - pb::sql::CommandPreparedStatementQuery command; - anyCommand.UnpackTo(&command); - return DoGetPreparedStatement(command, context, stream); + PreparedStatementQuery internal_command = + ParseCommandPreparedStatementQuery(anyCommand); + return DoGetPreparedStatement(internal_command, context, stream); } else if (anyCommand.Is()) { - pb::sql::CommandGetCatalogs command; - anyCommand.UnpackTo(&command); return DoGetCatalogs(context, stream); } else if (anyCommand.Is()) { - pb::sql::CommandGetSchemas command; - anyCommand.UnpackTo(&command); - return DoGetSchemas(command, context, stream); + GetSchemas internal_command = ParseCommandGetSchemas(anyCommand); + return DoGetSchemas(internal_command, context, stream); } else if (anyCommand.Is()) { - pb::sql::CommandGetTables command; - anyCommand.UnpackTo(&command); + GetTables command = ParseCommandGetTables(anyCommand); return DoGetTables(command, context, stream); } else if (anyCommand.Is()) { - pb::sql::CommandGetTableTypes command; - anyCommand.UnpackTo(&command); return DoGetTableTypes(context, stream); } else if (anyCommand.Is()) { - pb::sql::CommandGetSqlInfo command; - anyCommand.UnpackTo(&command); - return DoGetSqlInfo(command, context, stream); + GetSqlInfo internal_command = ParseCommandGetSqlInfo(anyCommand); + return DoGetSqlInfo(internal_command, context, stream); } else if (anyCommand.Is()) { - pb::sql::CommandGetPrimaryKeys command; - anyCommand.UnpackTo(&command); - return DoGetPrimaryKeys(command, context, stream); + GetPrimaryKeys internal_command = ParseCommandGetPrimaryKeys(anyCommand); + return DoGetPrimaryKeys(internal_command, context, stream); } else if (anyCommand.Is()) { - pb::sql::CommandGetExportedKeys command; - anyCommand.UnpackTo(&command); - return DoGetExportedKeys(command, context, stream); + GetExportedKeys internal_command = ParseCommandGetExportedKeys(anyCommand); + return DoGetExportedKeys(internal_command, context, stream); } else if (anyCommand.Is()) { - pb::sql::CommandGetImportedKeys command; - anyCommand.UnpackTo(&command); - return DoGetImportedKeys(command, context, stream); + GetImportedKeys internal_command = ParseCommandGetImportedKeys(anyCommand); + return DoGetImportedKeys(internal_command, context, stream); } else if (anyCommand.Is()) { - pb::sql::CommandGetCrossReference command; - anyCommand.UnpackTo(&command); - return DoGetCrossReference(command, context, stream); + GetCrossReference internal_command = ParseCommandGetCrossReference(anyCommand); + return DoGetCrossReference(internal_command, context, stream); } return Status::Invalid("The defined request is invalid."); @@ -145,17 +284,34 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size())); if (any.Is()) { - pb::sql::CommandStatementUpdate command; - any.UnpackTo(&command); - return DoPutCommandStatementUpdate(command, context, reader, writer); + StatementUpdate internal_command = ParseCommandStatementUpdate(any); + ARROW_ASSIGN_OR_RAISE(auto record_count, + DoPutCommandStatementUpdate(internal_command, context, reader)) + + pb::sql::DoPutUpdateResult result; + result.set_record_count(record_count); + + const std::shared_ptr& buffer = + Buffer::FromString(result.SerializeAsString()); + ARROW_RETURN_NOT_OK(writer->WriteMetadata(*buffer)); + + return Status::OK(); } else if (any.Is()) { - pb::sql::CommandPreparedStatementQuery command; - any.UnpackTo(&command); - return DoPutPreparedStatementQuery(command, context, reader, writer); + PreparedStatementQuery internal_command = ParseCommandPreparedStatementQuery(any); + return DoPutPreparedStatementQuery(internal_command, context, reader, writer); } else if (any.Is()) { - pb::sql::CommandPreparedStatementUpdate command; - any.UnpackTo(&command); - return DoPutPreparedStatementUpdate(command, context, reader, writer); + PreparedStatementUpdate internal_command = ParseCommandPreparedStatementUpdate(any); + ARROW_ASSIGN_OR_RAISE(auto record_count, + DoPutPreparedStatementUpdate(internal_command, context, reader)) + + pb::sql::DoPutUpdateResult result; + result.set_record_count(record_count); + + const std::shared_ptr& buffer = + Buffer::FromString(result.SerializeAsString()); + ARROW_RETURN_NOT_OK(writer->WriteMetadata(*buffer)); + + return Status::OK(); } return Status::Invalid("The defined request is invalid."); @@ -170,24 +326,44 @@ Status FlightSqlServerBase::ListActions(const ServerCallContext& context, Status FlightSqlServerBase::DoAction(const ServerCallContext& context, const Action& action, - std::unique_ptr* result) { + std::unique_ptr* result_stream) { if (action.type == FlightSqlServerBase::FLIGHT_SQL_CREATE_PREPARED_STATEMENT.type) { google::protobuf::Any anyCommand; anyCommand.ParseFromArray(action.body->data(), static_cast(action.body->size())); - pb::sql::ActionCreatePreparedStatementRequest command; - anyCommand.UnpackTo(&command); - - return CreatePreparedStatement(command, context, result); + ActionCreatePreparedStatementRequest internal_command = + ParseActionCreatePreparedStatementRequest(anyCommand); + ARROW_ASSIGN_OR_RAISE(auto result, CreatePreparedStatement(internal_command, context)) + + 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::FLIGHT_SQL_CLOSE_PREPARED_STATEMENT.type) { google::protobuf::Any anyCommand; anyCommand.ParseFromArray(action.body->data(), static_cast(action.body->size())); - pb::sql::ActionClosePreparedStatementRequest command; - anyCommand.UnpackTo(&command); + ActionClosePreparedStatementRequest internal_command = + ParseActionClosePreparedStatementRequest(anyCommand); - return ClosePreparedStatement(command, context, result); + return ClosePreparedStatement(internal_command, context, result_stream); } return Status::Invalid("The defined request is invalid."); } @@ -203,63 +379,65 @@ Status FlightSqlServerBase::DoGetCatalogs(const ServerCallContext& context, return Status::NotImplemented("DoGetCatalogs not implemented"); } -Status FlightSqlServerBase::GetFlightInfoStatement( - const pb::sql::CommandStatementQuery& command, const ServerCallContext& context, - const FlightDescriptor& descriptor, std::unique_ptr* info) { +Status FlightSqlServerBase::GetFlightInfoStatement(const StatementQuery& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoStatement not implemented"); } -Status FlightSqlServerBase::DoGetStatement(const pb::sql::TicketStatementQuery& command, +Status FlightSqlServerBase::DoGetStatement(const StatementQueryTicket& command, const ServerCallContext& context, std::unique_ptr* result) { return Status::NotImplemented("DoGetStatement not implemented"); } Status FlightSqlServerBase::GetFlightInfoPreparedStatement( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info) { + const PreparedStatementQuery& command, const ServerCallContext& context, + const FlightDescriptor& descriptor, std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoPreparedStatement not implemented"); } Status FlightSqlServerBase::DoGetPreparedStatement( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, std::unique_ptr* result) { + const PreparedStatementQuery& command, const ServerCallContext& context, + std::unique_ptr* result) { return Status::NotImplemented("DoGetPreparedStatement not implemented"); } -Status FlightSqlServerBase::GetFlightInfoSqlInfo( - const pb::sql::CommandGetSqlInfo& command, const ServerCallContext& context, - const FlightDescriptor& descriptor, std::unique_ptr* info) { +Status FlightSqlServerBase::GetFlightInfoSqlInfo(const GetSqlInfo& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoSqlInfo not implemented"); } -Status FlightSqlServerBase::DoGetSqlInfo(const pb::sql::CommandGetSqlInfo& command, +Status FlightSqlServerBase::DoGetSqlInfo(const GetSqlInfo& command, const ServerCallContext& context, std::unique_ptr* result) { return Status::NotImplemented("DoGetSqlInfo not implemented"); } -Status FlightSqlServerBase::GetFlightInfoSchemas( - const pb::sql::CommandGetSchemas& command, const ServerCallContext& context, - const FlightDescriptor& descriptor, std::unique_ptr* info) { +Status FlightSqlServerBase::GetFlightInfoSchemas(const GetSchemas& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoSchemas not implemented"); } -Status FlightSqlServerBase::DoGetSchemas(const pb::sql::CommandGetSchemas& command, +Status FlightSqlServerBase::DoGetSchemas(const GetSchemas& command, const ServerCallContext& context, std::unique_ptr* result) { return Status::NotImplemented("DoGetSchemas not implemented"); } -Status FlightSqlServerBase::GetFlightInfoTables(const pb::sql::CommandGetTables& command, +Status FlightSqlServerBase::GetFlightInfoTables(const GetTables& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoTables not implemented"); } -Status FlightSqlServerBase::DoGetTables(const pb::sql::CommandGetTables& command, +Status FlightSqlServerBase::DoGetTables(const GetTables& command, const ServerCallContext& context, std::unique_ptr* result) { return Status::NotImplemented("DoGetTables not implemented"); @@ -276,84 +454,86 @@ Status FlightSqlServerBase::DoGetTableTypes(const ServerCallContext& context, return Status::NotImplemented("DoGetTableTypes not implemented"); } -Status FlightSqlServerBase::GetFlightInfoPrimaryKeys( - const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, - const FlightDescriptor& descriptor, std::unique_ptr* info) { +Status FlightSqlServerBase::GetFlightInfoPrimaryKeys(const GetPrimaryKeys& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoPrimaryKeys not implemented"); } -Status FlightSqlServerBase::DoGetPrimaryKeys( - const pb::sql::CommandGetPrimaryKeys& command, const ServerCallContext& context, - std::unique_ptr* result) { +Status FlightSqlServerBase::DoGetPrimaryKeys(const GetPrimaryKeys& command, + const ServerCallContext& context, + std::unique_ptr* result) { return Status::NotImplemented("DoGetPrimaryKeys not implemented"); } -Status FlightSqlServerBase::GetFlightInfoExportedKeys( - const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, - const FlightDescriptor& descriptor, std::unique_ptr* info) { +Status FlightSqlServerBase::GetFlightInfoExportedKeys(const GetExportedKeys& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoExportedKeys not implemented"); } -Status FlightSqlServerBase::DoGetExportedKeys( - const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context, - std::unique_ptr* result) { +Status FlightSqlServerBase::DoGetExportedKeys(const GetExportedKeys& command, + const ServerCallContext& context, + std::unique_ptr* result) { return Status::NotImplemented("DoGetExportedKeys not implemented"); } -Status FlightSqlServerBase::GetFlightInfoImportedKeys( - const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, - const FlightDescriptor& descriptor, std::unique_ptr* info) { +Status FlightSqlServerBase::GetFlightInfoImportedKeys(const GetImportedKeys& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoImportedKeys not implemented"); } -Status FlightSqlServerBase::DoGetImportedKeys( - const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context, - std::unique_ptr* result) { +Status FlightSqlServerBase::DoGetImportedKeys(const GetImportedKeys& command, + const ServerCallContext& context, + std::unique_ptr* result) { return Status::NotImplemented("DoGetImportedKeys not implemented"); } Status FlightSqlServerBase::GetFlightInfoCrossReference( - const pb::sql::CommandGetCrossReference& command, const ServerCallContext& context, + const GetCrossReference& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoCrossReference not implemented"); } Status FlightSqlServerBase::DoGetCrossReference( - const pb::sql::CommandGetCrossReference& command, const ServerCallContext& context, + const GetCrossReference& command, const ServerCallContext& context, std::unique_ptr* result) { return Status::NotImplemented("DoGetCrossReference not implemented"); } -Status FlightSqlServerBase::CreatePreparedStatement( - const pb::sql::ActionCreatePreparedStatementRequest& request, - const ServerCallContext& context, std::unique_ptr* p_ptr) { +arrow::Result +FlightSqlServerBase::CreatePreparedStatement( + const ActionCreatePreparedStatementRequest& request, + const ServerCallContext& context) { return Status::NotImplemented("CreatePreparedStatement not implemented"); } Status FlightSqlServerBase::ClosePreparedStatement( - const pb::sql::ActionClosePreparedStatementRequest& request, - const ServerCallContext& context, std::unique_ptr* p_ptr) { + const ActionClosePreparedStatementRequest& request, const ServerCallContext& context, + std::unique_ptr* p_ptr) { return Status::NotImplemented("ClosePreparedStatement not implemented"); } Status FlightSqlServerBase::DoPutPreparedStatementQuery( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, std::unique_ptr& reader, + const PreparedStatementQuery& command, const ServerCallContext& context, + std::unique_ptr& reader, std::unique_ptr& writer) { return Status::NotImplemented("DoPutPreparedStatementQuery not implemented"); } -Status FlightSqlServerBase::DoPutPreparedStatementUpdate( - const pb::sql::CommandPreparedStatementUpdate& command, - const ServerCallContext& context, std::unique_ptr& reader, - std::unique_ptr& writer) { +arrow::Result FlightSqlServerBase::DoPutPreparedStatementUpdate( + const PreparedStatementUpdate& command, const ServerCallContext& context, + std::unique_ptr& reader) { return Status::NotImplemented("DoPutPreparedStatementUpdate not implemented"); } -Status FlightSqlServerBase::DoPutCommandStatementUpdate( - const pb::sql::CommandStatementUpdate& command, const ServerCallContext& context, - std::unique_ptr& reader, - std::unique_ptr& writer) { +arrow::Result FlightSqlServerBase::DoPutCommandStatementUpdate( + const StatementUpdate& command, const ServerCallContext& context, + std::unique_ptr& reader) { return Status::NotImplemented("DoPutCommandStatementUpdate not implemented"); } diff --git a/cpp/src/arrow/flight/flight-sql/server.h b/cpp/src/arrow/flight/flight-sql/server.h index f48fc7aae14..2bcfc6b5591 100644 --- a/cpp/src/arrow/flight/flight-sql/server.h +++ b/cpp/src/arrow/flight/flight-sql/server.h @@ -23,13 +23,109 @@ #include #include #include +#include -namespace pb = arrow::flight::protocol; +#include +#include + +#include "arrow/api.h" +#include "arrow/flight/flight-sql/example/sqlite_statement.h" +#include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/flight-sql/server.h" namespace arrow { namespace flight { namespace sql { +using StatementQuery = struct StatementQuery { std::string query; }; + +using StatementUpdate = struct StatementUpdate { std::string query; }; + +using StatementQueryTicket = struct StatementQueryTicket { + const std::string statement_handle; +}; + +using PreparedStatementQuery = struct PreparedStatementQuery { + const std::string prepared_statement_handle; +}; + +using PreparedStatementUpdate = struct PreparedStatementUpdate { + const std::string prepared_statement_handle; +}; + +using GetSqlInfo = struct GetSqlInfo { + // TODO: To be implemented. +}; + +using GetSchemas = struct GetSchemas { + const bool has_catalog; + const std::string catalog; + const bool has_schema_filter_pattern; + const std::string schema_filter_pattern; +}; + +using GetTables = struct GetTables { + const bool has_catalog; + const std::string catalog; + const bool has_schema_filter_pattern; + const std::string schema_filter_pattern; + const bool has_table_name_filter_pattern; + const std::string table_name_filter_pattern; + const std::vector table_types; + const bool include_schema; +}; + +using GetPrimaryKeys = struct GetPrimaryKeys { + const bool has_catalog; + const std::string catalog; + const bool has_schema; + const std::string schema; + const std::string table; +}; + +using GetExportedKeys = struct GetExportedKeys { + const bool has_catalog; + const std::string catalog; + const bool has_schema; + const std::string schema; + const std::string table; +}; + +using GetImportedKeys = struct GetImportedKeys { + const bool has_catalog; + const std::string catalog; + const bool has_schema; + const std::string schema; + const std::string table; +}; + +using GetCrossReference = struct GetCrossReference { + const bool has_pk_catalog; + const std::string pk_catalog; + const bool has_pk_schema; + const std::string pk_schema; + const std::string pk_table; + const bool has_fk_catalog; + const std::string fk_catalog; + const bool has_fk_schema; + const std::string fk_schema; + const std::string fk_table; +}; + +using ActionCreatePreparedStatementRequest = struct ActionCreatePreparedStatementRequest { + const std::string query; +}; + +using ActionClosePreparedStatementRequest = struct ActionClosePreparedStatementRequest { + const std::string prepared_statement_handle; +}; + +using ActionCreatePreparedStatementResult = struct ActionCreatePreparedStatementResult { + const std::shared_ptr dataset_schema; + const std::shared_ptr parameter_schema; + const std::string prepared_statement_handle; +}; + class FlightSqlServerBase : public FlightServerBase { public: Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, @@ -62,50 +158,49 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr* result) override; /// \brief Gets a FlightInfo for executing a SQL query. - /// \param[in] command The CommandStatementQuery object containing the SQL - /// statement. + /// \param[in] command The StatementQuery object containing the SQL statement. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status GetFlightInfoStatement(const pb::sql::CommandStatementQuery& command, + virtual Status GetFlightInfoStatement(const StatementQuery& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the query results. - /// \param[in] command The TicketStatementQuery containing the statement handle. + /// \param[in] command The StatementQueryTicket containing the statement handle. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetStatement(const pb::sql::TicketStatementQuery& command, + virtual Status DoGetStatement(const StatementQueryTicket& command, const ServerCallContext& context, std::unique_ptr* result); /// \brief Gets a FlightInfo for executing an already created prepared statement. - /// \param[in] command The CommandPreparedStatementQuery object containing the + /// \param[in] command The PreparedStatementQuery object containing the /// prepared statement handle. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoPreparedStatement( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual Status GetFlightInfoPreparedStatement(const PreparedStatementQuery& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the prepared statement query results. - /// \param[in] command The CommandPreparedStatementQuery object containing the + /// \param[in] command The PreparedStatementQuery object containing the /// prepared statement handle. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetPreparedStatement( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, std::unique_ptr* result); + virtual Status DoGetPreparedStatement(const PreparedStatementQuery& command, + const ServerCallContext& context, + std::unique_ptr* result); /// \brief Gets a FlightInfo for listing catalogs. /// \param[in] context Per-call context. @@ -124,69 +219,66 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr* result); /// \brief Gets a FlightInfo for retrieving other information (See SqlInfo). - /// \param[in] command The CommandGetSqlInfo object containing the list of SqlInfo + /// \param[in] command The GetSqlInfo object containing the list of SqlInfo /// to be returned. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status GetFlightInfoSqlInfo(const pb::sql::CommandGetSqlInfo& command, + virtual Status GetFlightInfoSqlInfo(const GetSqlInfo& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the list of SqlInfo results. - /// \param[in] command The CommandGetSqlInfo object containing the list of SqlInfo + /// \param[in] command The GetSqlInfo object containing the list of SqlInfo /// to be returned. /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetSqlInfo(const pb::sql::CommandGetSqlInfo& command, - const ServerCallContext& context, + virtual Status DoGetSqlInfo(const GetSqlInfo& command, const ServerCallContext& context, std::unique_ptr* result); /// \brief Gets a FlightInfo for listing schemas. - /// \param[in] command The CommandGetSchemas object which may contain filters for + /// \param[in] command The GetSchemas object which may contain filters for /// catalog and schema name. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status GetFlightInfoSchemas(const pb::sql::CommandGetSchemas& command, + virtual Status GetFlightInfoSchemas(const GetSchemas& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the list of schemas. - /// \param[in] command The CommandGetSchemas object which may contain filters for + /// \param[in] command The GetSchemas object which may contain filters for /// catalog and schema name. /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetSchemas(const pb::sql::CommandGetSchemas& command, - const ServerCallContext& context, + virtual Status DoGetSchemas(const GetSchemas& command, const ServerCallContext& context, std::unique_ptr* result); ///\brief Gets a FlightInfo for listing tables. - /// \param[in] command The CommandGetTables object which may contain filters for + /// \param[in] command The GetTables object which may contain filters for /// catalog, schema and table names. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status GetFlightInfoTables(const pb::sql::CommandGetTables& command, + virtual Status GetFlightInfoTables(const GetTables& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the list of tables. - /// \param[in] command The CommandGetTables object which may contain filters for + /// \param[in] command The GetTables object which may contain filters for /// catalog, schema and table names. /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetTables(const pb::sql::CommandGetTables& command, - const ServerCallContext& context, + virtual Status DoGetTables(const GetTables& command, const ServerCallContext& context, std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about the table types. @@ -207,14 +299,14 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about primary and foreign keys. - /// \param[in] command The CommandGetPrimaryKeys object with necessary information + /// \param[in] command The GetPrimaryKeys object with necessary information /// to execute the request. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, + virtual Status GetFlightInfoPrimaryKeys(const GetPrimaryKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info); @@ -222,24 +314,24 @@ class FlightSqlServerBase : public FlightServerBase { /// \brief Gets a FlightDataStream containing the data related to the primary and /// foreign /// keys. - /// \param[in] command The CommandGetPrimaryKeys object with necessary information + /// \param[in] command The GetPrimaryKeys object with necessary information /// to execute the request. /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys& command, + virtual Status DoGetPrimaryKeys(const GetPrimaryKeys& command, const ServerCallContext& context, std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about foreign and primary keys. - /// \param[in] command The CommandGetExportedKeys object with necessary information + /// \param[in] command The GetExportedKeys object with necessary information /// to execute the request. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoExportedKeys(const pb::sql::CommandGetExportedKeys& command, + virtual Status GetFlightInfoExportedKeys(const GetExportedKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info); @@ -247,41 +339,41 @@ class FlightSqlServerBase : public FlightServerBase { /// \brief Gets a FlightDataStream containing the data related to the foreign and /// primary /// keys. - /// \param[in] command The CommandGetExportedKeys object with necessary information + /// \param[in] command The GetExportedKeys object with necessary information /// to execute the request. /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetExportedKeys(const pb::sql::CommandGetExportedKeys& command, + virtual Status DoGetExportedKeys(const GetExportedKeys& command, const ServerCallContext& context, std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about foreign and primary keys. - /// \param[in] command The CommandGetImportedKeys object with necessary information + /// \param[in] command The GetImportedKeys object with necessary information /// to execute the request. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoImportedKeys(const pb::sql::CommandGetImportedKeys& command, + virtual Status GetFlightInfoImportedKeys(const GetImportedKeys& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the data related to the foreign and /// primary keys. - /// \param[in] command The CommandGetImportedKeys object with necessary information + /// \param[in] command The GetImportedKeys object with necessary information /// to execute the request. /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetImportedKeys(const pb::sql::CommandGetImportedKeys& command, + virtual Status DoGetImportedKeys(const GetImportedKeys& command, const ServerCallContext& context, std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about foreign and primary keys. - /// \param[in] command The CommandGetCrossReference object with necessary + /// \param[in] command The GetCrossReference object with necessary /// information /// to execute the request. /// \param[in] context Per-call context. @@ -289,40 +381,40 @@ class FlightSqlServerBase : public FlightServerBase { /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoCrossReference( - const pb::sql::CommandGetCrossReference& command, const ServerCallContext& context, - const FlightDescriptor& descriptor, std::unique_ptr* info); + virtual Status GetFlightInfoCrossReference(const GetCrossReference& command, + const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the data related to the foreign and /// primary keys. - /// \param[in] command The CommandGetCrossReference object with necessary information + /// \param[in] command The GetCrossReference object with necessary information /// to execute the request. /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetCrossReference(const pb::sql::CommandGetCrossReference& command, + virtual Status DoGetCrossReference(const GetCrossReference& command, const ServerCallContext& context, std::unique_ptr* result); /// \brief Executes an update SQL statement. - /// \param[in] command The CommandStatementUpdate object containing the SQL statement. + /// \param[in] command The StatementUpdate object containing the SQL statement. /// \param[in] context The call context. /// \param[in] reader a sequence of uploaded record batches. - /// \param[in] writer send metadata back to the client. - /// \return Status. - virtual Status DoPutCommandStatementUpdate( - const pb::sql::CommandStatementUpdate& command, const ServerCallContext& context, - std::unique_ptr& reader, - std::unique_ptr& writer); + /// \return The changed record count. + virtual arrow::Result DoPutCommandStatementUpdate( + const StatementUpdate& command, const ServerCallContext& context, + std::unique_ptr& reader); /// \brief Create a prepared statement from given SQL statement. /// \param[in] request The ActionCreatePreparedStatementRequest object containing the /// SQL statement. /// \param[in] context The call context. - /// \param[out] result ResultStream containing a ActionCreatePreparedStatementResult. - virtual Status CreatePreparedStatement( - const pb::sql::ActionCreatePreparedStatementRequest& request, - const ServerCallContext& context, std::unique_ptr* result); + /// \return A ActionCreatePreparedStatementResult containing the dataset + /// and parameter schemas and a handle for created statement. + virtual arrow::Result CreatePreparedStatement( + const ActionCreatePreparedStatementRequest& request, + const ServerCallContext& context); /// \brief Closes a prepared statement. /// \param[in] request The ActionClosePreparedStatementRequest object containing the @@ -330,72 +422,73 @@ class FlightSqlServerBase : public FlightServerBase { /// \param[in] context The call context. /// \param[out] result Empty ResultStream. virtual Status ClosePreparedStatement( - const pb::sql::ActionClosePreparedStatementRequest& request, + const ActionClosePreparedStatementRequest& request, const ServerCallContext& context, std::unique_ptr* result); /// \brief Binds parameters to given prepared statement. - /// \param[in] command The CommandPreparedStatementQuery object containing the + /// \param[in] command The PreparedStatementQuery object containing the /// prepared statement handle. /// \param[in] context The call context. /// \param[in] reader A sequence of uploaded record batches. /// \param[in] writer Send metadata back to the client. virtual Status DoPutPreparedStatementQuery( - const pb::sql::CommandPreparedStatementQuery& command, - const ServerCallContext& context, std::unique_ptr& reader, + const PreparedStatementQuery& command, const ServerCallContext& context, + std::unique_ptr& reader, std::unique_ptr& writer); /// \brief Executes an update SQL prepared statement. - /// \param[in] command The CommandPreparedStatementUpdate object containing the + /// \param[in] command The PreparedStatementUpdate object containing the /// prepared statement handle. /// \param[in] context The call context. /// \param[in] reader a sequence of uploaded record batches. - /// \param[in] writer send metadata back to the client. - /// \return Status. - virtual Status DoPutPreparedStatementUpdate( - const pb::sql::CommandPreparedStatementUpdate& command, - const ServerCallContext& context, std::unique_ptr& reader, - std::unique_ptr& writer); + /// \return The changed record count. + virtual arrow::Result DoPutPreparedStatementUpdate( + const PreparedStatementUpdate& command, const ServerCallContext& context, + std::unique_ptr& reader); + + protected: + static std::string CreateStatementQueryTicket(const std::string& statement_handle); }; /// \brief Auxiliary class containing all Schemas used on Flight SQL. class SqlSchema { public: - /// \brief Gets the Schema used on CommandGetCatalogs response. + /// \brief Gets the Schema used on GetCatalogs response. /// \return The default schema template. static std::shared_ptr GetCatalogsSchema(); - /// \brief Gets the Schema used on CommandGetSchemas response. + /// \brief Gets the Schema used on GetSchemas response. /// \return The default schema template. static std::shared_ptr GetSchemasSchema(); - /// \brief Gets the Schema used on CommandGetTables response when included schema + /// \brief Gets 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 Gets the Schema used on CommandGetTables response when included schema + /// \brief Gets 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 Gets the Schema used on CommandGetTableTypes response. + /// \brief Gets the Schema used on GetTableTypes response. /// \return The default schema template. static std::shared_ptr GetTableTypesSchema(); - /// \brief Gets the Schema used on CommandGetPrimaryKeys response when included schema + /// \brief Gets 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 Gets the Schema used on CommandGetImportedKeys response. + /// \brief Gets the Schema used on GetImportedKeys response. /// \return The default schema template. static std::shared_ptr GetExportedKeysSchema(); - /// \brief Gets the Schema used on CommandGetImportedKeys response. + /// \brief Gets the Schema used on GetImportedKeys response. /// \return The default schema template. static std::shared_ptr GetImportedKeysSchema(); - /// \brief Gets the Schema used on CommandGetCrossReference response. + /// \brief Gets the Schema used on GetCrossReference response. /// \return The default schema template. static std::shared_ptr GetCrossReferenceSchema(); }; From 5acfa2325674dd61a31e6146e5e29289d1ffbd72 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 26 Oct 2021 11:29:48 -0300 Subject: [PATCH 161/237] Fix compiler warnings on client_impl.h --- cpp/src/arrow/flight/flight-sql/client_impl.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index ae76ee51379..5e1cdbd55bb 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -55,7 +55,7 @@ FlightSqlClientT::~FlightSqlClientT() = default; template PreparedStatementT::~PreparedStatementT() { - Close(); + ARROW_UNUSED(Close()); } inline FlightDescriptor GetFlightDescriptorForCommand( @@ -320,7 +320,7 @@ arrow::Result> PreparedStatementT::Execute() { if (parameter_binding && parameter_binding->num_rows() > 0) { std::unique_ptr writer; std::unique_ptr reader; - client->DoPut(options, descriptor, parameter_binding->schema(), &writer, &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()); From 30929b7920d477c2a9f2d7d69dc5d9253c4349f2 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 26 Oct 2021 11:55:35 -0300 Subject: [PATCH 162/237] Fix linting issues --- .../arrow/flight/flight-sql/CMakeLists.txt | 34 ++--- cpp/src/arrow/flight/flight-sql/client.h | 2 +- cpp/src/arrow/flight/flight-sql/client_impl.h | 12 +- cpp/src/arrow/flight/flight-sql/server.cc | 125 ++++++++++-------- cpp/src/arrow/flight/flight-sql/server.h | 90 ++++++------- 5 files changed, 142 insertions(+), 121 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 7491896fe9d..22258990de2 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -75,14 +75,14 @@ target_link_libraries(flight_sql_test_app PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES}) add_arrow_test(flight_sql_test - SOURCES - client_test.cc - server_test.cc - "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" - STATIC_LINK_LIBS - ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} - LABELS - "arrow_flight_sql") + SOURCES + client_test.cc + server_test.cc + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" + STATIC_LINK_LIBS + ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} + LABELS + "arrow_flight_sql") # Build test server for unit tests if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) @@ -90,16 +90,16 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) include_directories(${SQLite3_INCLUDE_DIRS}) add_executable(flight_sql_test_server - server.cc - test_server_cli.cc - example/sqlite_statement.cc - example/sqlite_statement_batch_reader.cc - example/sqlite_server.cc - example/sqlite_tables_schema_batch_reader.cpp - "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc") + server.cc + test_server_cli.cc + example/sqlite_statement.cc + example/sqlite_statement_batch_reader.cc + example/sqlite_server.cc + example/sqlite_tables_schema_batch_reader.cpp + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc") target_link_libraries(flight_sql_test_server - PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES} - ${SQLite3_LIBRARIES}) + PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES} + ${SQLite3_LIBRARIES}) if(ARROW_BUILD_TESTS) add_dependencies(arrow-flight-sql-test flight_sql_test_server) diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index f69d9c561c2..078c3d84b82 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -47,7 +47,7 @@ class ARROW_EXPORT PreparedStatementT { PreparedStatementT( T* client, const std::string& query, pb::sql::ActionCreatePreparedStatementResult& prepared_statement_result, - const FlightCallOptions& options); + FlightCallOptions options); /// \brief Default destructor for the PreparedStatement class. /// The destructor will call the Close method from the class in order, diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 5e1cdbd55bb..67e56dec89c 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -43,12 +43,11 @@ template PreparedStatementT::PreparedStatementT( T* client_, const std::string& query, pb::sql::ActionCreatePreparedStatementResult& prepared_statement_result_, - const FlightCallOptions& options_) + FlightCallOptions options_) : client(client_), - options(options_), - prepared_statement_result(std::move(prepared_statement_result_)) { - is_closed = false; -} + options(std::move(options_)), + prepared_statement_result(std::move(prepared_statement_result_)), + is_closed(false) {} template FlightSqlClientT::~FlightSqlClientT() = default; @@ -320,7 +319,8 @@ arrow::Result> PreparedStatementT::Execute() { 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(client->DoPut(options, descriptor, parameter_binding->schema(), + &writer, &reader)); ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*parameter_binding)); ARROW_RETURN_NOT_OK(writer->DoneWriting()); diff --git a/cpp/src/arrow/flight/flight-sql/server.cc b/cpp/src/arrow/flight/flight-sql/server.cc index b8682e947d1..5b4580592da 100644 --- a/cpp/src/arrow/flight/flight-sql/server.cc +++ b/cpp/src/arrow/flight/flight-sql/server.cc @@ -18,7 +18,7 @@ // Interfaces to use for defining Flight RPC servers. API should be considered // experimental for now -#include "server.h" +#include "arrow/flight/flight-sql/server.h" #include #include @@ -49,68 +49,77 @@ GetCrossReference ParseCommandGetCrossReference(const google::protobuf::Any& any pb::sql::CommandGetCrossReference command; any.UnpackTo(&command); - return (GetCrossReference){.has_pk_catalog = command.has_pk_catalog(), - .pk_catalog = command.pk_catalog(), - .has_pk_schema = command.has_pk_schema(), - .pk_schema = command.pk_schema(), - .pk_table = command.pk_table(), - .has_fk_catalog = command.has_fk_catalog(), - .fk_catalog = command.fk_catalog(), - .has_fk_schema = command.has_fk_schema(), - .fk_schema = command.fk_schema(), - .fk_table = command.fk_table()}; + GetCrossReference result; + result.has_pk_catalog = command.has_pk_catalog(); + result.pk_catalog = command.pk_catalog(); + result.has_pk_schema = command.has_pk_schema(); + result.pk_schema = command.pk_schema(); + result.pk_table = command.pk_table(); + result.has_fk_catalog = command.has_fk_catalog(); + result.fk_catalog = command.fk_catalog(); + result.has_fk_schema = command.has_fk_schema(); + result.fk_schema = command.fk_schema(); + result.fk_table = command.fk_table(); + return result; } GetImportedKeys ParseCommandGetImportedKeys(const google::protobuf::Any& any) { pb::sql::CommandGetImportedKeys command; any.UnpackTo(&command); - return (GetImportedKeys){.has_catalog = command.has_catalog(), - .catalog = command.catalog(), - .has_schema = command.has_schema(), - .schema = command.schema(), - .table = command.table()}; + GetImportedKeys result; + result.has_catalog = command.has_catalog(); + result.catalog = command.catalog(); + result.has_schema = command.has_schema(); + result.schema = command.schema(); + result.table = command.table(); + return result; } GetExportedKeys ParseCommandGetExportedKeys(const google::protobuf::Any& any) { pb::sql::CommandGetExportedKeys command; any.UnpackTo(&command); - return (GetExportedKeys){.has_catalog = command.has_catalog(), - .catalog = command.catalog(), - .has_schema = command.has_schema(), - .schema = command.schema(), - .table = command.table()}; + GetExportedKeys result; + result.has_catalog = command.has_catalog(); + result.catalog = command.catalog(); + result.has_schema = command.has_schema(); + result.schema = command.schema(); + result.table = command.table(); + return result; } GetPrimaryKeys ParseCommandGetPrimaryKeys(const google::protobuf::Any& any) { pb::sql::CommandGetPrimaryKeys command; any.UnpackTo(&command); - return (GetPrimaryKeys){.has_catalog = command.has_catalog(), - .catalog = command.catalog(), - .has_schema = command.has_schema(), - .schema = command.schema(), - .table = command.table()}; + GetPrimaryKeys result; + result.has_catalog = command.has_catalog(); + result.catalog = command.catalog(); + result.has_schema = command.has_schema(); + result.schema = command.schema(); + result.table = command.table(); + return result; } GetSqlInfo ParseCommandGetSqlInfo(const google::protobuf::Any& any) { pb::sql::CommandGetSqlInfo command; any.UnpackTo(&command); - return (GetSqlInfo){}; + GetSqlInfo result; + return result; } GetSchemas ParseCommandGetSchemas(const google::protobuf::Any& any) { pb::sql::CommandGetSchemas command; any.UnpackTo(&command); - return (GetSchemas){ - .has_catalog = command.has_catalog(), - .catalog = command.catalog(), - .has_schema_filter_pattern = command.has_schema_filter_pattern(), - .schema_filter_pattern = command.schema_filter_pattern(), - }; + GetSchemas result; + result.has_catalog = command.has_catalog(); + result.catalog = command.catalog(); + result.has_schema_filter_pattern = command.has_schema_filter_pattern(); + result.schema_filter_pattern = command.schema_filter_pattern(); + return result; } PreparedStatementQuery ParseCommandPreparedStatementQuery( @@ -118,15 +127,18 @@ PreparedStatementQuery ParseCommandPreparedStatementQuery( pb::sql::CommandPreparedStatementQuery command; any.UnpackTo(&command); - return (PreparedStatementQuery){.prepared_statement_handle = - command.prepared_statement_handle()}; + PreparedStatementQuery result; + result.prepared_statement_handle = command.prepared_statement_handle(); + return result; } StatementQuery ParseCommandStatementQuery(const google::protobuf::Any& any) { pb::sql::CommandStatementQuery command; any.UnpackTo(&command); - return (StatementQuery){.query = command.query()}; + StatementQuery result; + result.query = command.query(); + return result; } GetTables ParseCommandGetTables(const google::protobuf::Any& anyCommand) { @@ -138,29 +150,34 @@ GetTables ParseCommandGetTables(const google::protobuf::Any& anyCommand) { for (const auto& item : command.table_types()) { table_types.push_back(item); } - return (GetTables){ - .has_catalog = command.has_catalog(), - .catalog = command.catalog(), - .has_schema_filter_pattern = command.has_schema_filter_pattern(), - .schema_filter_pattern = command.schema_filter_pattern(), - .has_table_name_filter_pattern = command.has_table_name_filter_pattern(), - .table_name_filter_pattern = command.table_name_filter_pattern(), - .table_types = table_types, - .include_schema = command.include_schema()}; + GetTables result; + result.has_catalog = command.has_catalog(); + result.catalog = command.catalog(); + result.has_schema_filter_pattern = command.has_schema_filter_pattern(); + result.schema_filter_pattern = command.schema_filter_pattern(); + result.has_table_name_filter_pattern = command.has_table_name_filter_pattern(); + result.table_name_filter_pattern = command.table_name_filter_pattern(); + result.table_types = table_types; + result.include_schema = command.include_schema(); + return result; } StatementQueryTicket ParseStatementQueryTicket(const google::protobuf::Any& anyCommand) { pb::sql::TicketStatementQuery command; anyCommand.UnpackTo(&command); - return (StatementQueryTicket){.statement_handle = command.statement_handle()}; + StatementQueryTicket result; + result.statement_handle = command.statement_handle(); + return result; } StatementUpdate ParseCommandStatementUpdate(const google::protobuf::Any& any) { pb::sql::CommandStatementUpdate command; any.UnpackTo(&command); - return (StatementUpdate){.query = command.query()}; + StatementUpdate result; + result.query = command.query(); + return result; } PreparedStatementUpdate ParseCommandPreparedStatementUpdate( @@ -168,8 +185,9 @@ PreparedStatementUpdate ParseCommandPreparedStatementUpdate( pb::sql::CommandPreparedStatementUpdate command; any.UnpackTo(&command); - return (PreparedStatementUpdate){.prepared_statement_handle = - command.prepared_statement_handle()}; + PreparedStatementUpdate result; + result.prepared_statement_handle = command.prepared_statement_handle(); + return result; } ActionCreatePreparedStatementRequest ParseActionCreatePreparedStatementRequest( @@ -177,7 +195,9 @@ ActionCreatePreparedStatementRequest ParseActionCreatePreparedStatementRequest( pb::sql::ActionCreatePreparedStatementRequest command; anyCommand.UnpackTo(&command); - return (ActionCreatePreparedStatementRequest){.query = command.query()}; + ActionCreatePreparedStatementRequest result; + result.query = command.query(); + return result; } ActionClosePreparedStatementRequest ParseActionClosePreparedStatementRequest( @@ -185,8 +205,9 @@ ActionClosePreparedStatementRequest ParseActionClosePreparedStatementRequest( pb::sql::ActionClosePreparedStatementRequest command; anyCommand.UnpackTo(&command); - return (ActionClosePreparedStatementRequest){.prepared_statement_handle = - command.prepared_statement_handle()}; + ActionClosePreparedStatementRequest result; + result.prepared_statement_handle = command.prepared_statement_handle(); + return result; } Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, diff --git a/cpp/src/arrow/flight/flight-sql/server.h b/cpp/src/arrow/flight/flight-sql/server.h index 2bcfc6b5591..0c5b4657c3f 100644 --- a/cpp/src/arrow/flight/flight-sql/server.h +++ b/cpp/src/arrow/flight/flight-sql/server.h @@ -42,15 +42,15 @@ using StatementQuery = struct StatementQuery { std::string query; }; using StatementUpdate = struct StatementUpdate { std::string query; }; using StatementQueryTicket = struct StatementQueryTicket { - const std::string statement_handle; + std::string statement_handle; }; using PreparedStatementQuery = struct PreparedStatementQuery { - const std::string prepared_statement_handle; + std::string prepared_statement_handle; }; using PreparedStatementUpdate = struct PreparedStatementUpdate { - const std::string prepared_statement_handle; + std::string prepared_statement_handle; }; using GetSqlInfo = struct GetSqlInfo { @@ -58,72 +58,72 @@ using GetSqlInfo = struct GetSqlInfo { }; using GetSchemas = struct GetSchemas { - const bool has_catalog; - const std::string catalog; - const bool has_schema_filter_pattern; - const std::string schema_filter_pattern; + bool has_catalog; + std::string catalog; + bool has_schema_filter_pattern; + std::string schema_filter_pattern; }; using GetTables = struct GetTables { - const bool has_catalog; - const std::string catalog; - const bool has_schema_filter_pattern; - const std::string schema_filter_pattern; - const bool has_table_name_filter_pattern; - const std::string table_name_filter_pattern; - const std::vector table_types; - const bool include_schema; + bool has_catalog; + std::string catalog; + bool has_schema_filter_pattern; + std::string schema_filter_pattern; + bool has_table_name_filter_pattern; + std::string table_name_filter_pattern; + std::vector table_types; + bool include_schema; }; using GetPrimaryKeys = struct GetPrimaryKeys { - const bool has_catalog; - const std::string catalog; - const bool has_schema; - const std::string schema; - const std::string table; + bool has_catalog; + std::string catalog; + bool has_schema; + std::string schema; + std::string table; }; using GetExportedKeys = struct GetExportedKeys { - const bool has_catalog; - const std::string catalog; - const bool has_schema; - const std::string schema; - const std::string table; + bool has_catalog; + std::string catalog; + bool has_schema; + std::string schema; + std::string table; }; using GetImportedKeys = struct GetImportedKeys { - const bool has_catalog; - const std::string catalog; - const bool has_schema; - const std::string schema; - const std::string table; + bool has_catalog; + std::string catalog; + bool has_schema; + std::string schema; + std::string table; }; using GetCrossReference = struct GetCrossReference { - const bool has_pk_catalog; - const std::string pk_catalog; - const bool has_pk_schema; - const std::string pk_schema; - const std::string pk_table; - const bool has_fk_catalog; - const std::string fk_catalog; - const bool has_fk_schema; - const std::string fk_schema; - const std::string fk_table; + bool has_pk_catalog; + std::string pk_catalog; + bool has_pk_schema; + std::string pk_schema; + std::string pk_table; + bool has_fk_catalog; + std::string fk_catalog; + bool has_fk_schema; + std::string fk_schema; + std::string fk_table; }; using ActionCreatePreparedStatementRequest = struct ActionCreatePreparedStatementRequest { - const std::string query; + std::string query; }; using ActionClosePreparedStatementRequest = struct ActionClosePreparedStatementRequest { - const std::string prepared_statement_handle; + std::string prepared_statement_handle; }; using ActionCreatePreparedStatementResult = struct ActionCreatePreparedStatementResult { - const std::shared_ptr dataset_schema; - const std::shared_ptr parameter_schema; - const std::string prepared_statement_handle; + std::shared_ptr dataset_schema; + std::shared_ptr parameter_schema; + std::string prepared_statement_handle; }; class FlightSqlServerBase : public FlightServerBase { From 6ab6e27e2262a5bc535e075747693e97740ee190 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Tue, 26 Oct 2021 13:49:46 -0300 Subject: [PATCH 163/237] Change ASSERT_TRUE to AsserTableEqual() --- .../arrow/flight/flight-sql/server_test.cc | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/server_test.cc b/cpp/src/arrow/flight/flight-sql/server_test.cc index 89145967eed..0ceff978c51 100644 --- a/cpp/src/arrow/flight/flight-sql/server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/server_test.cc @@ -93,7 +93,7 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { const std::shared_ptr
& expected_table = Table::Make( expected_schema, {id_array, keyname_array, value_array, foreignId_array}); - ASSERT_TRUE(expected_table->Equals(*table)); + AssertTablesEqual(*expected_table, *table); } TEST(TestFlightSqlServer, TestCommandGetTables) { @@ -119,7 +119,7 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { const std::shared_ptr
& expected_table = Table::Make( SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); - ASSERT_TRUE(expected_table->Equals(*table)); + AssertTablesEqual(*expected_table, *table); } TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { @@ -146,7 +146,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { const std::shared_ptr
& expected_table = Table::Make( SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); - ASSERT_TRUE(expected_table->Equals(*table)); + AssertTablesEqual(*expected_table, *table); } TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { @@ -196,7 +196,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { const std::shared_ptr
& expected_table = Table::Make( SqlSchema::GetTablesSchema(), {catalog_name, schema_name, table_name, table_type}); - ASSERT_TRUE(expected_table->Equals(*table)); + AssertTablesEqual(*expected_table, *table); } TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { @@ -239,7 +239,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { Table::Make(SqlSchema::GetTablesSchemaWithIncludedSchema(), {catalog_name, schema_name, table_name, table_type, table_schema}); - ASSERT_TRUE(expected_table->Equals(*table)); + AssertTablesEqual(*expected_table, *table); } TEST(TestFlightSqlServer, TestCommandGetCatalogs) { @@ -286,7 +286,7 @@ TEST(TestFlightSqlServer, TestCommandGetTableTypes) { const std::shared_ptr
& expected_table = Table::Make(SqlSchema::GetTableTypesSchema(), {table_type}); - ASSERT_TRUE(expected_table->Equals(*table)); + AssertTablesEqual(*expected_table, *table); } TEST(TestFlightSqlServer, TestCommandStatementUpdate) { @@ -339,7 +339,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQuery) { const std::shared_ptr
& expected_table = Table::Make( expected_schema, {id_array, keyname_array, value_array, foreignId_array}); - ASSERT_TRUE(expected_table->Equals(*table)); + AssertTablesEqual(*expected_table, *table); } TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) { @@ -403,7 +403,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) const std::shared_ptr
& expected_table = Table::Make( expected_schema, {id_array, keyname_array, value_array, foreignId_array}); - ASSERT_TRUE(expected_table->Equals(*table)); + AssertTablesEqual(*expected_table, *table); } arrow::Result ExecuteCountQuery(const std::string& query) { @@ -529,7 +529,7 @@ TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { SqlSchema::GetPrimaryKeysSchema(), {catalog_name, schema_name, table_name, column_name, key_sequence, key_name}); - ASSERT_TRUE(expected_table->Equals(*table)); + AssertTablesEqual(*expected_table, *table); } TEST(TestFlightSqlServer, TestCommandGetImportedKeys) { @@ -569,7 +569,7 @@ TEST(TestFlightSqlServer, TestCommandGetImportedKeys) { {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}); - ASSERT_TRUE(expected_table->Equals(*table)); + AssertTablesEqual(*expected_table, *table); } TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { @@ -609,7 +609,7 @@ TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { {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}); - ASSERT_TRUE(expected_table->Equals(*table)); + AssertTablesEqual(*expected_table, *table); } TEST(TestFlightSqlServer, TestCommandGetCrossReference) { @@ -649,7 +649,7 @@ TEST(TestFlightSqlServer, TestCommandGetCrossReference) { {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}); - ASSERT_TRUE(expected_table->Equals(*table)); + AssertTablesEqual(*expected_table, *table); } auto env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); From 92b7b48a346f46d4f0049ea6479a29609ade7399 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 26 Oct 2021 17:52:14 -0300 Subject: [PATCH 164/237] Improve DoPutUpdateResult parsing --- cpp/src/arrow/flight/flight-sql/client_impl.h | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index 67e56dec89c..ebaf9faf187 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -106,12 +106,10 @@ arrow::Result FlightSqlClientT::ExecuteUpdate( pb::sql::DoPutUpdateResult doPutUpdateResult; - Buffer* pBuffer = metadata.get(); + pb::sql::DoPutUpdateResult result; + result.ParseFromArray(metadata->data(), static_cast(metadata->size())); - const std::string& string = pBuffer->ToString(); - - doPutUpdateResult.ParseFrom(string); - return doPutUpdateResult.record_count(); + return result.record_count(); } template @@ -365,12 +363,10 @@ arrow::Result PreparedStatementT::ExecuteUpdate() { ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata)); ARROW_RETURN_NOT_OK(writer->Close()); - pb::sql::DoPutUpdateResult doPutUpdateResult; - const std::string& metadataAsString = metadata->ToString(); - - doPutUpdateResult.ParseFrom(metadataAsString); + pb::sql::DoPutUpdateResult result; + result.ParseFromArray(metadata->data(), static_cast(metadata->size())); - return doPutUpdateResult.record_count(); + return result.record_count(); } template From 4dfa02606adbea3947696024d9fc50a6336bbce9 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 27 Oct 2021 13:17:40 -0300 Subject: [PATCH 165/237] Handle errors on all Parse and Unpack calls to Protobuf (#185) --- cpp/src/arrow/flight/flight-sql/client_impl.h | 18 +- cpp/src/arrow/flight/flight-sql/server.cc | 203 +++++++++++------- 2 files changed, 141 insertions(+), 80 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client_impl.h index ebaf9faf187..33139424358 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client_impl.h @@ -107,7 +107,9 @@ arrow::Result FlightSqlClientT::ExecuteUpdate( pb::sql::DoPutUpdateResult doPutUpdateResult; pb::sql::DoPutUpdateResult result; - result.ParseFromArray(metadata->data(), static_cast(metadata->size())); + if (!result.ParseFromArray(metadata->data(), static_cast(metadata->size()))) { + return Status::Invalid("Unable to parse DoPutUpdateResult object."); + } return result.record_count(); } @@ -286,12 +288,16 @@ arrow::Result>> FlightSqlClientT::Prepa google::protobuf::Any prepared_result; std::shared_ptr message = std::move(result->body); - - prepared_result.ParseFromArray(message->data(), static_cast(message->size())); + if (!prepared_result.ParseFromArray(message->data(), + static_cast(message->size()))) { + return Status::Invalid("Unable to parse packed ActionCreatePreparedStatementResult"); + } pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; - prepared_result.UnpackTo(&prepared_statement_result); + if (!prepared_result.UnpackTo(&prepared_statement_result)) { + return Status::Invalid("Unable to unpack ActionCreatePreparedStatementResult"); + } return std::shared_ptr>( new PreparedStatementT(client.get(), query, prepared_statement_result, options)); @@ -364,7 +370,9 @@ arrow::Result PreparedStatementT::ExecuteUpdate() { ARROW_RETURN_NOT_OK(writer->Close()); pb::sql::DoPutUpdateResult result; - result.ParseFromArray(metadata->data(), static_cast(metadata->size())); + if (!result.ParseFromArray(metadata->data(), static_cast(metadata->size()))) { + return Status::Invalid("Unable to parse DoPutUpdateResult object."); + } return result.record_count(); } diff --git a/cpp/src/arrow/flight/flight-sql/server.cc b/cpp/src/arrow/flight/flight-sql/server.cc index 5b4580592da..469f74d9456 100644 --- a/cpp/src/arrow/flight/flight-sql/server.cc +++ b/cpp/src/arrow/flight/flight-sql/server.cc @@ -45,9 +45,12 @@ namespace sql { namespace pb = arrow::flight::protocol; -GetCrossReference ParseCommandGetCrossReference(const google::protobuf::Any& any) { +arrow::Result ParseCommandGetCrossReference( + const google::protobuf::Any& any) { pb::sql::CommandGetCrossReference command; - any.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetCrossReference."); + } GetCrossReference result; result.has_pk_catalog = command.has_pk_catalog(); @@ -63,9 +66,12 @@ GetCrossReference ParseCommandGetCrossReference(const google::protobuf::Any& any return result; } -GetImportedKeys ParseCommandGetImportedKeys(const google::protobuf::Any& any) { +arrow::Result ParseCommandGetImportedKeys( + const google::protobuf::Any& any) { pb::sql::CommandGetImportedKeys command; - any.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetImportedKeys."); + } GetImportedKeys result; result.has_catalog = command.has_catalog(); @@ -76,9 +82,12 @@ GetImportedKeys ParseCommandGetImportedKeys(const google::protobuf::Any& any) { return result; } -GetExportedKeys ParseCommandGetExportedKeys(const google::protobuf::Any& any) { +arrow::Result ParseCommandGetExportedKeys( + const google::protobuf::Any& any) { pb::sql::CommandGetExportedKeys command; - any.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetExportedKeys."); + } GetExportedKeys result; result.has_catalog = command.has_catalog(); @@ -89,9 +98,12 @@ GetExportedKeys ParseCommandGetExportedKeys(const google::protobuf::Any& any) { return result; } -GetPrimaryKeys ParseCommandGetPrimaryKeys(const google::protobuf::Any& any) { +arrow::Result ParseCommandGetPrimaryKeys( + const google::protobuf::Any& any) { pb::sql::CommandGetPrimaryKeys command; - any.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetPrimaryKeys."); + } GetPrimaryKeys result; result.has_catalog = command.has_catalog(); @@ -102,17 +114,21 @@ GetPrimaryKeys ParseCommandGetPrimaryKeys(const google::protobuf::Any& any) { return result; } -GetSqlInfo ParseCommandGetSqlInfo(const google::protobuf::Any& any) { +arrow::Result ParseCommandGetSqlInfo(const google::protobuf::Any& any) { pb::sql::CommandGetSqlInfo command; - any.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetSqlInfo."); + } GetSqlInfo result; return result; } -GetSchemas ParseCommandGetSchemas(const google::protobuf::Any& any) { +arrow::Result ParseCommandGetSchemas(const google::protobuf::Any& any) { pb::sql::CommandGetSchemas command; - any.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetSchemas."); + } GetSchemas result; result.has_catalog = command.has_catalog(); @@ -122,28 +138,35 @@ GetSchemas ParseCommandGetSchemas(const google::protobuf::Any& any) { return result; } -PreparedStatementQuery ParseCommandPreparedStatementQuery( +arrow::Result ParseCommandPreparedStatementQuery( const google::protobuf::Any& any) { pb::sql::CommandPreparedStatementQuery command; - any.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandPreparedStatementQuery."); + } PreparedStatementQuery result; result.prepared_statement_handle = command.prepared_statement_handle(); return result; } -StatementQuery ParseCommandStatementQuery(const google::protobuf::Any& any) { +arrow::Result ParseCommandStatementQuery( + const google::protobuf::Any& any) { pb::sql::CommandStatementQuery command; - any.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandStatementQuery."); + } StatementQuery result; result.query = command.query(); return result; } -GetTables ParseCommandGetTables(const google::protobuf::Any& anyCommand) { +arrow::Result ParseCommandGetTables(const google::protobuf::Any& any) { pb::sql::CommandGetTables command; - anyCommand.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandGetTables."); + } std::vector table_types; table_types.reserve(command.table_types_size()); @@ -162,48 +185,60 @@ GetTables ParseCommandGetTables(const google::protobuf::Any& anyCommand) { return result; } -StatementQueryTicket ParseStatementQueryTicket(const google::protobuf::Any& anyCommand) { +arrow::Result ParseStatementQueryTicket( + const google::protobuf::Any& any) { pb::sql::TicketStatementQuery command; - anyCommand.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack TicketStatementQuery."); + } StatementQueryTicket result; result.statement_handle = command.statement_handle(); return result; } -StatementUpdate ParseCommandStatementUpdate(const google::protobuf::Any& any) { +arrow::Result ParseCommandStatementUpdate( + const google::protobuf::Any& any) { pb::sql::CommandStatementUpdate command; - any.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandStatementUpdate."); + } StatementUpdate result; result.query = command.query(); return result; } -PreparedStatementUpdate ParseCommandPreparedStatementUpdate( +arrow::Result ParseCommandPreparedStatementUpdate( const google::protobuf::Any& any) { pb::sql::CommandPreparedStatementUpdate command; - any.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandPreparedStatementUpdate."); + } PreparedStatementUpdate result; result.prepared_statement_handle = command.prepared_statement_handle(); return result; } -ActionCreatePreparedStatementRequest ParseActionCreatePreparedStatementRequest( - const google::protobuf::Any& anyCommand) { +arrow::Result +ParseActionCreatePreparedStatementRequest(const google::protobuf::Any& any) { pb::sql::ActionCreatePreparedStatementRequest command; - anyCommand.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack ActionCreatePreparedStatementRequest."); + } ActionCreatePreparedStatementRequest result; result.query = command.query(); return result; } -ActionClosePreparedStatementRequest ParseActionClosePreparedStatementRequest( - const google::protobuf::Any& anyCommand) { +arrow::Result +ParseActionClosePreparedStatementRequest(const google::protobuf::Any& any) { pb::sql::ActionClosePreparedStatementRequest command; - anyCommand.UnpackTo(&command); + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack ActionClosePreparedStatementRequest."); + } ActionClosePreparedStatementRequest result; result.prepared_statement_handle = command.prepared_statement_handle(); @@ -214,38 +249,46 @@ Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, std::unique_ptr* info) { google::protobuf::Any any; - any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size())); + if (!any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size()))) { + return Status::Invalid("Unable to parse command"); + } if (any.Is()) { - StatementQuery internal_command = ParseCommandStatementQuery(any); + ARROW_ASSIGN_OR_RAISE(StatementQuery internal_command, + ParseCommandStatementQuery(any)); return GetFlightInfoStatement(internal_command, context, request, info); } else if (any.Is()) { - PreparedStatementQuery internal_command = ParseCommandPreparedStatementQuery(any); + ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command, + ParseCommandPreparedStatementQuery(any)); return GetFlightInfoPreparedStatement(internal_command, context, request, info); } else if (any.Is()) { return GetFlightInfoCatalogs(context, request, info); } else if (any.Is()) { - GetSchemas internal_command = ParseCommandGetSchemas(any); + ARROW_ASSIGN_OR_RAISE(GetSchemas internal_command, ParseCommandGetSchemas(any)); return GetFlightInfoSchemas(internal_command, context, request, info); } else if (any.Is()) { - GetTables command = ParseCommandGetTables(any); + ARROW_ASSIGN_OR_RAISE(GetTables command, ParseCommandGetTables(any)); return GetFlightInfoTables(command, context, request, info); } else if (any.Is()) { return GetFlightInfoTableTypes(context, request, info); } else if (any.Is()) { - GetSqlInfo internal_command = ParseCommandGetSqlInfo(any); + ARROW_ASSIGN_OR_RAISE(GetSqlInfo internal_command, ParseCommandGetSqlInfo(any)); return GetFlightInfoSqlInfo(internal_command, context, request, info); } else if (any.Is()) { - GetPrimaryKeys internal_command = ParseCommandGetPrimaryKeys(any); + ARROW_ASSIGN_OR_RAISE(GetPrimaryKeys internal_command, + ParseCommandGetPrimaryKeys(any)); return GetFlightInfoPrimaryKeys(internal_command, context, request, info); } else if (any.Is()) { - GetExportedKeys internal_command = ParseCommandGetExportedKeys(any); + ARROW_ASSIGN_OR_RAISE(GetExportedKeys internal_command, + ParseCommandGetExportedKeys(any)); return GetFlightInfoExportedKeys(internal_command, context, request, info); } else if (any.Is()) { - GetImportedKeys internal_command = ParseCommandGetImportedKeys(any); + ARROW_ASSIGN_OR_RAISE(GetImportedKeys internal_command, + ParseCommandGetImportedKeys(any)); return GetFlightInfoImportedKeys(internal_command, context, request, info); } else if (any.Is()) { - GetCrossReference internal_command = ParseCommandGetCrossReference(any); + ARROW_ASSIGN_OR_RAISE(GetCrossReference internal_command, + ParseCommandGetCrossReference(any)); return GetFlightInfoCrossReference(internal_command, context, request, info); } @@ -254,42 +297,48 @@ Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket& request, std::unique_ptr* stream) { - google::protobuf::Any anyCommand; + google::protobuf::Any any; - anyCommand.ParseFromArray(request.ticket.data(), - static_cast(request.ticket.size())); + if (!any.ParseFromArray(request.ticket.data(), + static_cast(request.ticket.size()))) { + return Status::Invalid("Unable to parse ticket."); + } - if (anyCommand.Is()) { - StatementQueryTicket command = ParseStatementQueryTicket(anyCommand); + if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(StatementQueryTicket command, ParseStatementQueryTicket(any)); return DoGetStatement(command, context, stream); - } else if (anyCommand.Is()) { - PreparedStatementQuery internal_command = - ParseCommandPreparedStatementQuery(anyCommand); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command, + ParseCommandPreparedStatementQuery(any)); return DoGetPreparedStatement(internal_command, context, stream); - } else if (anyCommand.Is()) { + } else if (any.Is()) { return DoGetCatalogs(context, stream); - } else if (anyCommand.Is()) { - GetSchemas internal_command = ParseCommandGetSchemas(anyCommand); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetSchemas internal_command, ParseCommandGetSchemas(any)); return DoGetSchemas(internal_command, context, stream); - } else if (anyCommand.Is()) { - GetTables command = ParseCommandGetTables(anyCommand); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetTables command, ParseCommandGetTables(any)); return DoGetTables(command, context, stream); - } else if (anyCommand.Is()) { + } else if (any.Is()) { return DoGetTableTypes(context, stream); - } else if (anyCommand.Is()) { - GetSqlInfo internal_command = ParseCommandGetSqlInfo(anyCommand); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetSqlInfo internal_command, ParseCommandGetSqlInfo(any)); return DoGetSqlInfo(internal_command, context, stream); - } else if (anyCommand.Is()) { - GetPrimaryKeys internal_command = ParseCommandGetPrimaryKeys(anyCommand); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetPrimaryKeys internal_command, + ParseCommandGetPrimaryKeys(any)); return DoGetPrimaryKeys(internal_command, context, stream); - } else if (anyCommand.Is()) { - GetExportedKeys internal_command = ParseCommandGetExportedKeys(anyCommand); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetExportedKeys internal_command, + ParseCommandGetExportedKeys(any)); return DoGetExportedKeys(internal_command, context, stream); - } else if (anyCommand.Is()) { - GetImportedKeys internal_command = ParseCommandGetImportedKeys(anyCommand); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetImportedKeys internal_command, + ParseCommandGetImportedKeys(any)); return DoGetImportedKeys(internal_command, context, stream); - } else if (anyCommand.Is()) { - GetCrossReference internal_command = ParseCommandGetCrossReference(anyCommand); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(GetCrossReference internal_command, + ParseCommandGetCrossReference(any)); return DoGetCrossReference(internal_command, context, stream); } @@ -305,7 +354,8 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size())); if (any.Is()) { - StatementUpdate internal_command = ParseCommandStatementUpdate(any); + ARROW_ASSIGN_OR_RAISE(StatementUpdate internal_command, + ParseCommandStatementUpdate(any)); ARROW_ASSIGN_OR_RAISE(auto record_count, DoPutCommandStatementUpdate(internal_command, context, reader)) @@ -318,10 +368,12 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, return Status::OK(); } else if (any.Is()) { - PreparedStatementQuery internal_command = ParseCommandPreparedStatementQuery(any); + ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command, + ParseCommandPreparedStatementQuery(any)); return DoPutPreparedStatementQuery(internal_command, context, reader, writer); } else if (any.Is()) { - PreparedStatementUpdate internal_command = ParseCommandPreparedStatementUpdate(any); + ARROW_ASSIGN_OR_RAISE(PreparedStatementUpdate internal_command, + ParseCommandPreparedStatementUpdate(any)); ARROW_ASSIGN_OR_RAISE(auto record_count, DoPutPreparedStatementUpdate(internal_command, context, reader)) @@ -349,11 +401,12 @@ Status FlightSqlServerBase::DoAction(const ServerCallContext& context, const Action& action, std::unique_ptr* result_stream) { if (action.type == FlightSqlServerBase::FLIGHT_SQL_CREATE_PREPARED_STATEMENT.type) { - google::protobuf::Any anyCommand; - anyCommand.ParseFromArray(action.body->data(), static_cast(action.body->size())); + google::protobuf::Any any_command; + any_command.ParseFromArray(action.body->data(), + static_cast(action.body->size())); - ActionCreatePreparedStatementRequest internal_command = - ParseActionCreatePreparedStatementRequest(anyCommand); + ARROW_ASSIGN_OR_RAISE(ActionCreatePreparedStatementRequest internal_command, + ParseActionCreatePreparedStatementRequest(any_command)); ARROW_ASSIGN_OR_RAISE(auto result, CreatePreparedStatement(internal_command, context)) pb::sql::ActionCreatePreparedStatementResult action_result; @@ -378,11 +431,11 @@ Status FlightSqlServerBase::DoAction(const ServerCallContext& context, return Status::OK(); } else if (action.type == FlightSqlServerBase::FLIGHT_SQL_CLOSE_PREPARED_STATEMENT.type) { - google::protobuf::Any anyCommand; - anyCommand.ParseFromArray(action.body->data(), static_cast(action.body->size())); + google::protobuf::Any any; + any.ParseFromArray(action.body->data(), static_cast(action.body->size())); - ActionClosePreparedStatementRequest internal_command = - ParseActionClosePreparedStatementRequest(anyCommand); + ARROW_ASSIGN_OR_RAISE(ActionClosePreparedStatementRequest internal_command, + ParseActionClosePreparedStatementRequest(any)); return ClosePreparedStatement(internal_command, context, result_stream); } From f1199debe277b0629692d27d0a409b016b6107df Mon Sep 17 00:00:00 2001 From: Jose Almeida <53087160+jcralmeida@users.noreply.github.com> Date: Thu, 28 Oct 2021 16:56:28 -0300 Subject: [PATCH 166/237] Add ARROW_EXPORT and change designated initializer from struct (#189) --- cpp/src/arrow/flight/flight-sql/server.h | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/server.h b/cpp/src/arrow/flight/flight-sql/server.h index 0c5b4657c3f..f754fdde0da 100644 --- a/cpp/src/arrow/flight/flight-sql/server.h +++ b/cpp/src/arrow/flight/flight-sql/server.h @@ -139,15 +139,13 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr writer) override; const ActionType FLIGHT_SQL_CREATE_PREPARED_STATEMENT = - ActionType{.type = "CreatePreparedStatement", - .description = - "Creates a reusable prepared statement resource on the server.\n" + ActionType{"CreatePreparedStatement", + "Creates a reusable prepared statement resource on the server.\n" "Request Message: ActionCreatePreparedStatementRequest\n" "Response Message: ActionCreatePreparedStatementResult"}; const ActionType FLIGHT_SQL_CLOSE_PREPARED_STATEMENT = - ActionType{.type = "ClosePreparedStatement", - .description = - "Closes a reusable prepared statement resource on the server.\n" + ActionType{"ClosePreparedStatement", + "Closes a reusable prepared statement resource on the server.\n" "Request Message: ActionClosePreparedStatementRequest\n" "Response Message: N/A"}; @@ -451,7 +449,7 @@ class FlightSqlServerBase : public FlightServerBase { }; /// \brief Auxiliary class containing all Schemas used on Flight SQL. -class SqlSchema { +class ARROW_EXPORT SqlSchema { public: /// \brief Gets the Schema used on GetCatalogs response. /// \return The default schema template. From 3d0dcfecb203fe89354a0422592873a17bc8ca92 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 28 Oct 2021 17:07:26 -0300 Subject: [PATCH 167/237] [C++] Remove templating from client interface (#184) * Fix variable initialization * Remove templating from FlightSqlClient and PreparedStatement classes * Fix inconsistent member names * Make PreparedStatement an inner class of FlightSqlClient * WIP: Use references on FlightCLientImpl methods * Use shared_ptr to avoid dangling pointers * Log error when deleting PreparedStatement Co-authored-by: Jose Almeida --- .../arrow/flight/flight-sql/CMakeLists.txt | 15 +- .../flight-sql/{client_impl.h => client.cc} | 207 ++++++++---------- cpp/src/arrow/flight/flight-sql/client.h | 129 ++++++----- .../flight/flight-sql/client_internal.cc | 94 ++++++++ .../arrow/flight/flight-sql/client_internal.h | 59 +++++ .../arrow/flight/flight-sql/client_test.cc | 123 ++++++----- .../flight/flight-sql/example/sqlite_server.h | 1 + cpp/src/arrow/flight/flight-sql/server.cc | 3 + cpp/src/arrow/flight/flight-sql/server.h | 2 - .../arrow/flight/flight-sql/server_test.cc | 3 +- .../arrow/flight/flight-sql/test_app_cli.cc | 3 +- .../flight/flight-sql/test_server_cli.cc | 2 - 12 files changed, 401 insertions(+), 240 deletions(-) rename cpp/src/arrow/flight/flight-sql/{client_impl.h => client.cc} (64%) create mode 100644 cpp/src/arrow/flight/flight-sql/client_internal.cc create mode 100644 cpp/src/arrow/flight/flight-sql/client_internal.h diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt index 22258990de2..ce634f1513b 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight-sql/CMakeLists.txt @@ -36,7 +36,7 @@ set_source_files_properties(${FLIGHT_SQL_GENERATED_PROTO_FILES} PROPERTIES GENER add_custom_target(flight_sql_protobuf_gen ALL DEPENDS ${FLIGHT_SQL_GENERATED_PROTO_FILES}) -set(ARROW_FLIGHT_SQL_SRCS server.cc) +set(ARROW_FLIGHT_SQL_SRCS server.cc client.cc client_internal.cc) add_arrow_lib(arrow_flight_sql CMAKE_PACKAGE_NAME @@ -74,10 +74,19 @@ add_executable(flight_sql_test_app test_app_cli.cc target_link_libraries(flight_sql_test_app PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES}) -add_arrow_test(flight_sql_test +add_arrow_test(flight_sql_client_test SOURCES client_test.cc + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" + STATIC_LINK_LIBS + ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} + LABELS + "arrow_flight_sql") + +add_arrow_test(flight_sql_server_test + SOURCES server_test.cc + client_internal.cc "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" STATIC_LINK_LIBS ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} @@ -102,6 +111,6 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) ${SQLite3_LIBRARIES}) if(ARROW_BUILD_TESTS) - add_dependencies(arrow-flight-sql-test flight_sql_test_server) + add_dependencies(arrow-flight-sql-server-test flight_sql_test_server) endif() endif() diff --git a/cpp/src/arrow/flight/flight-sql/client_impl.h b/cpp/src/arrow/flight/flight-sql/client.cc similarity index 64% rename from cpp/src/arrow/flight/flight-sql/client_impl.h rename to cpp/src/arrow/flight/flight-sql/client.cc index 33139424358..ed7e7f4defa 100644 --- a/cpp/src/arrow/flight/flight-sql/client_impl.h +++ b/cpp/src/arrow/flight/flight-sql/client.cc @@ -17,44 +17,45 @@ #include #include +#include +#include #include #include #include #include #include +#include #include -#include +#include #include -namespace pb = arrow::flight::protocol; - namespace arrow { namespace flight { namespace sql { -namespace internal { -template -FlightSqlClientT::FlightSqlClientT(std::unique_ptr& client) { - this->client = std::move(client); -} +FlightSqlClient::FlightSqlClient(std::shared_ptr client) + : impl_(std::move(client)) {} + +FlightSqlClient::FlightSqlClient(std::unique_ptr client) + : impl_(internal::FlightClientImpl_Create(std::move(client))) {} + +FlightSqlClient::PreparedStatement::PreparedStatement( + std::shared_ptr client, const std::string& query, + pb::sql::ActionCreatePreparedStatementResult& prepared_statement_result, + FlightCallOptions options) + : client_(client), + options_(std::move(options)), + prepared_statement_result_(std::move(prepared_statement_result)), + is_closed_(false) {} + +FlightSqlClient::~FlightSqlClient() = default; -template -PreparedStatementT::PreparedStatementT( - T* client_, const std::string& query, - pb::sql::ActionCreatePreparedStatementResult& prepared_statement_result_, - FlightCallOptions options_) - : client(client_), - options(std::move(options_)), - prepared_statement_result(std::move(prepared_statement_result_)), - is_closed(false) {} - -template -FlightSqlClientT::~FlightSqlClientT() = default; - -template -PreparedStatementT::~PreparedStatementT() { - ARROW_UNUSED(Close()); +FlightSqlClient::PreparedStatement::~PreparedStatement() { + const Status status = Close(); + if (!status.ok()) { + ARROW_LOG(ERROR) << "Failed to delete PreparedStatement: " << status.ToString(); + } } inline FlightDescriptor GetFlightDescriptorForCommand( @@ -66,30 +67,28 @@ inline FlightDescriptor GetFlightDescriptorForCommand( return FlightDescriptor::Command(string); } -template arrow::Result> GetFlightInfoForCommand( - const std::unique_ptr& client, const FlightCallOptions& options, + internal::FlightClientImpl& client, const FlightCallOptions& options, const google::protobuf::Message& command) { const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); std::unique_ptr flight_info; - ARROW_RETURN_NOT_OK(client->GetFlightInfo(options, descriptor, &flight_info)); + ARROW_RETURN_NOT_OK(internal::FlightClientImpl_GetFlightInfo(client, options, + descriptor, &flight_info)); return std::move(flight_info); } -template -arrow::Result> FlightSqlClientT::Execute( +arrow::Result> FlightSqlClient::Execute( const FlightCallOptions& options, const std::string& query) const { pb::sql::CommandStatementQuery command; command.set_query(query); - return GetFlightInfoForCommand(client, options, command); + return GetFlightInfoForCommand(*impl_, options, command); } -template -arrow::Result FlightSqlClientT::ExecuteUpdate( - const FlightCallOptions& options, const std::string& query) const { +arrow::Result FlightSqlClient::ExecuteUpdate(const FlightCallOptions& options, + const std::string& query) const { pb::sql::CommandStatementUpdate command; command.set_query(query); @@ -98,11 +97,12 @@ arrow::Result FlightSqlClientT::ExecuteUpdate( std::unique_ptr writer; std::unique_ptr reader; - ARROW_RETURN_NOT_OK(client->DoPut(options, descriptor, NULLPTR, &writer, &reader)); + ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoPut(*impl_, options, descriptor, + NULLPTR, &writer, &reader)); std::shared_ptr metadata; - const Status& status = reader->ReadMetadata(&metadata); + ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata)); pb::sql::DoPutUpdateResult doPutUpdateResult; @@ -114,16 +114,14 @@ arrow::Result FlightSqlClientT::ExecuteUpdate( return result.record_count(); } -template -arrow::Result> FlightSqlClientT::GetCatalogs( +arrow::Result> FlightSqlClient::GetCatalogs( const FlightCallOptions& options) const { pb::sql::CommandGetCatalogs command; - return GetFlightInfoForCommand(client, options, command); + return GetFlightInfoForCommand(*impl_, options, command); } -template -arrow::Result> FlightSqlClientT::GetSchemas( +arrow::Result> FlightSqlClient::GetSchemas( const FlightCallOptions& options, const std::string* catalog, const std::string* schema_filter_pattern) const { pb::sql::CommandGetSchemas command; @@ -134,11 +132,10 @@ arrow::Result> FlightSqlClientT::GetSchemas( command.set_schema_filter_pattern(*schema_filter_pattern); } - return GetFlightInfoForCommand(client, options, command); + return GetFlightInfoForCommand(*impl_, options, command); } -template -arrow::Result> FlightSqlClientT::GetTables( +arrow::Result> FlightSqlClient::GetTables( const FlightCallOptions& options, const std::string* catalog, const std::string* schema_filter_pattern, const std::string* table_filter_pattern, bool include_schema, std::vector& table_types) const { @@ -162,11 +159,10 @@ arrow::Result> FlightSqlClientT::GetTables( command.add_table_types(table_type); } - return GetFlightInfoForCommand(client, options, command); + return GetFlightInfoForCommand(*impl_, options, command); } -template -arrow::Result> FlightSqlClientT::GetPrimaryKeys( +arrow::Result> FlightSqlClient::GetPrimaryKeys( const FlightCallOptions& options, const std::string* catalog, const std::string* schema, const std::string& table) const { pb::sql::CommandGetPrimaryKeys command; @@ -181,11 +177,10 @@ arrow::Result> FlightSqlClientT::GetPrimaryKeys( command.set_table(table); - return GetFlightInfoForCommand(client, options, command); + return GetFlightInfoForCommand(*impl_, options, command); } -template -arrow::Result> FlightSqlClientT::GetExportedKeys( +arrow::Result> FlightSqlClient::GetExportedKeys( const FlightCallOptions& options, const std::string* catalog, const std::string* schema, const std::string& table) const { pb::sql::CommandGetExportedKeys command; @@ -200,11 +195,10 @@ arrow::Result> FlightSqlClientT::GetExportedKeys( command.set_table(table); - return GetFlightInfoForCommand(client, options, command); + return GetFlightInfoForCommand(*impl_, options, command); } -template -arrow::Result> FlightSqlClientT::GetImportedKeys( +arrow::Result> FlightSqlClient::GetImportedKeys( const FlightCallOptions& options, const std::string* catalog, const std::string* schema, const std::string& table) const { pb::sql::CommandGetImportedKeys command; @@ -219,11 +213,10 @@ arrow::Result> FlightSqlClientT::GetImportedKeys( command.set_table(table); - return GetFlightInfoForCommand(client, options, command); + return GetFlightInfoForCommand(*impl_, options, command); } -template -arrow::Result> FlightSqlClientT::GetCrossReference( +arrow::Result> FlightSqlClient::GetCrossReference( const FlightCallOptions& options, const std::string* pk_catalog, const std::string* pk_schema, const std::string& pk_table, const std::string* fk_catalog, const std::string* fk_schema, @@ -246,29 +239,26 @@ arrow::Result> FlightSqlClientT::GetCrossReferenc } command.set_fk_table(fk_table); - return GetFlightInfoForCommand(client, options, command); + return GetFlightInfoForCommand(*impl_, options, command); } -template -arrow::Result> FlightSqlClientT::GetTableTypes( +arrow::Result> FlightSqlClient::GetTableTypes( const FlightCallOptions& options) const { pb::sql::CommandGetTableTypes command; - return GetFlightInfoForCommand(client, options, command); + return GetFlightInfoForCommand(*impl_, options, command); } -template -arrow::Result> FlightSqlClientT::DoGet( +arrow::Result> FlightSqlClient::DoGet( const FlightCallOptions& options, const Ticket& ticket) const { std::unique_ptr stream; - ARROW_RETURN_NOT_OK(client->DoGet(options, ticket, &stream)); + ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoGet(*impl_, options, ticket, &stream)); return std::move(stream); } -template -arrow::Result>> FlightSqlClientT::Prepare( - const FlightCallOptions& options, const std::string& query) { +arrow::Result> +FlightSqlClient::Prepare(const FlightCallOptions& options, const std::string& query) { google::protobuf::Any command; pb::sql::ActionCreatePreparedStatementRequest request; request.set_query(query); @@ -280,7 +270,8 @@ arrow::Result>> FlightSqlClientT::Prepa std::unique_ptr results; - ARROW_RETURN_NOT_OK(client->DoAction(options, action, &results)); + ARROW_RETURN_NOT_OK( + internal::FlightClientImpl_DoAction(*impl_, options, action, &results)); std::unique_ptr result; ARROW_RETURN_NOT_OK(results->Next(&result)); @@ -299,20 +290,19 @@ arrow::Result>> FlightSqlClientT::Prepa return Status::Invalid("Unable to unpack ActionCreatePreparedStatementResult"); } - return std::shared_ptr>( - new PreparedStatementT(client.get(), query, prepared_statement_result, options)); + return std::make_shared(impl_, query, prepared_statement_result, + options); } -template -arrow::Result> PreparedStatementT::Execute() { - if (is_closed) { +arrow::Result> FlightSqlClient::PreparedStatement::Execute() { + if (is_closed_) { return Status::Invalid("Statement already closed."); } pb::sql::CommandPreparedStatementQuery execute_query_command; execute_query_command.set_prepared_statement_handle( - prepared_statement_result.prepared_statement_handle()); + prepared_statement_result_.prepared_statement_handle()); google::protobuf::Any any; any.PackFrom(execute_query_command); @@ -320,13 +310,13 @@ arrow::Result> PreparedStatementT::Execute() { const std::string& string = any.SerializeAsString(); const FlightDescriptor descriptor = FlightDescriptor::Command(string); - if (parameter_binding && parameter_binding->num_rows() > 0) { + 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(internal::FlightClientImpl_DoPut( + *client_, options_, descriptor, parameter_binding_->schema(), &writer, &reader)); - ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*parameter_binding)); + 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; @@ -334,31 +324,32 @@ arrow::Result> PreparedStatementT::Execute() { } std::unique_ptr info; - ARROW_RETURN_NOT_OK(client->GetFlightInfo(options, descriptor, &info)); + ARROW_RETURN_NOT_OK( + internal::FlightClientImpl_GetFlightInfo(*client_, options_, descriptor, &info)); return std::move(info); } -template -arrow::Result PreparedStatementT::ExecuteUpdate() { - if (is_closed) { +arrow::Result FlightSqlClient::PreparedStatement::ExecuteUpdate() { + if (is_closed_) { return Status::Invalid("Statement already closed."); } pb::sql::CommandPreparedStatementUpdate command; command.set_prepared_statement_handle( - prepared_statement_result.prepared_statement_handle()); + prepared_statement_result_.prepared_statement_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)); + if (parameter_binding_ && parameter_binding_->num_rows() > 0) { + ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoPut( + *client_, 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)); + ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoPut(*client_, options_, descriptor, + schema, &writer, &reader)); const auto& record_batch = arrow::RecordBatch::Make(schema, 0, (std::vector>){}); ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*record_batch)); @@ -377,22 +368,18 @@ arrow::Result PreparedStatementT::ExecuteUpdate() { return result.record_count(); } -template -Status PreparedStatementT::SetParameters( - std::shared_ptr parameter_binding_) { - parameter_binding = std::move(parameter_binding_); +Status FlightSqlClient::PreparedStatement::SetParameters( + std::shared_ptr parameter_binding) { + parameter_binding_ = std::move(parameter_binding); return Status::OK(); } -template -bool PreparedStatementT::IsClosed() const { - return is_closed; -} +bool FlightSqlClient::PreparedStatement::IsClosed() const { return is_closed_; } -template -arrow::Result> PreparedStatementT::GetResultSetSchema() { - auto& args = prepared_statement_result.dataset_schema(); +arrow::Result> +FlightSqlClient::PreparedStatement::GetResultSetSchema() { + auto& args = prepared_statement_result_.dataset_schema(); std::shared_ptr schema_buffer = std::make_shared(args); io::BufferReader reader(schema_buffer); @@ -401,9 +388,9 @@ arrow::Result> PreparedStatementT::GetResultSetSchema return ReadSchema(&reader, &in_memo); } -template -arrow::Result> PreparedStatementT::GetParameterSchema() { - auto& args = prepared_statement_result.parameter_schema(); +arrow::Result> +FlightSqlClient::PreparedStatement::GetParameterSchema() { + auto& args = prepared_statement_result_.parameter_schema(); std::shared_ptr schema_buffer = std::make_shared(args); io::BufferReader reader(schema_buffer); @@ -412,15 +399,14 @@ arrow::Result> PreparedStatementT::GetParameterSchema return ReadSchema(&reader, &in_memo); } -template -Status PreparedStatementT::Close() { - if (is_closed) { +Status FlightSqlClient::PreparedStatement::Close() { + if (is_closed_) { return Status::Invalid("Statement already closed."); } google::protobuf::Any command; pb::sql::ActionClosePreparedStatementRequest request; request.set_prepared_statement_handle( - prepared_statement_result.prepared_statement_handle()); + prepared_statement_result_.prepared_statement_handle()); command.PackFrom(request); @@ -430,30 +416,27 @@ Status PreparedStatementT::Close() { std::unique_ptr results; - ARROW_RETURN_NOT_OK(client->DoAction(options, action, &results)); + ARROW_RETURN_NOT_OK(FlightClientImpl_DoAction(*client_, options_, action, &results)); - is_closed = true; + is_closed_ = true; return Status::OK(); } -template -arrow::Result> FlightSqlClientT::GetSqlInfo( +arrow::Result> FlightSqlClient::GetSqlInfo( const FlightCallOptions& options, const std::vector& sql_info) const { pb::sql::CommandGetSqlInfo command; for (const int& info : sql_info) command.add_info(info); - return GetFlightInfoForCommand(client, options, command); + return GetFlightInfoForCommand(*impl_, options, command); } -template -arrow::Result> FlightSqlClientT::GetSqlInfo( +arrow::Result> FlightSqlClient::GetSqlInfo( const FlightCallOptions& options, const std::vector& sql_info) const { return GetSqlInfo(options, reinterpret_cast&>(sql_info)); } -} // namespace internal } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 078c3d84b82..0cba9714406 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -30,68 +30,22 @@ namespace pb = arrow::flight::protocol; namespace arrow { namespace flight { namespace sql { + namespace internal { +class FlightClientImpl; +} -template -class ARROW_EXPORT PreparedStatementT { - T* client; - FlightCallOptions options; - pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; - std::shared_ptr parameter_binding; - bool is_closed; +/// \brief Flight client with Flight SQL semantics. +class ARROW_EXPORT FlightSqlClient { + private: + std::shared_ptr impl_; public: - /// \brief Constructor for the PreparedStatement class. - /// \param[in] client A raw pointer to FlightClient. - /// \param[in] query The query that will be executed. - PreparedStatementT( - T* client, const std::string& query, - pb::sql::ActionCreatePreparedStatementResult& prepared_statement_result, - 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. - ~PreparedStatementT(); - - /// \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. - arrow::Result> GetParameterSchema(); - - /// \brief Retrieve the ResultSet schema from the query. - /// \return The ResultSet schema from the query. - arrow::Result> GetResultSetSchema(); - - /// \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 Closes the prepared statement. - /// \param[in] options RPC-layer hints for this call. - /// \return Status. - Status Close(); - - /// \brief Checks if the prepared statement is closed. - /// \return The state of the prepared statement. - bool IsClosed() const; -}; + class PreparedStatement; -/// \brief Flight client with Flight SQL semantics. -template -class FlightSqlClientT { - public: - explicit FlightSqlClientT(std::unique_ptr& client); + explicit FlightSqlClient(std::shared_ptr client); - ~FlightSqlClientT(); + ~FlightSqlClient(); /// \brief Execute a query on the server. /// \param[in] options RPC-layer hints for this call. @@ -217,20 +171,63 @@ class FlightSqlClientT { /// \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( + arrow::Result> Prepare( const FlightCallOptions& options, const std::string& query); - private: - std::unique_ptr client; + explicit FlightSqlClient(std::unique_ptr client); + + class PreparedStatement { + std::shared_ptr client_; + FlightCallOptions options_; + pb::sql::ActionCreatePreparedStatementResult prepared_statement_result_; + std::shared_ptr parameter_binding_; + bool is_closed_; + + public: + /// \brief Constructor for the PreparedStatement class. + /// \param[in] query The query that will be executed. + PreparedStatement( + std::shared_ptr client_, const std::string& query, + pb::sql::ActionCreatePreparedStatementResult& prepared_statement_result, + 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. + ~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. + arrow::Result> GetParameterSchema(); + + /// \brief Retrieve the ResultSet schema from the query. + /// \return The ResultSet schema from the query. + arrow::Result> GetResultSetSchema(); + + /// \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 Closes the prepared statement. + /// \param[in] options RPC-layer hints for this call. + /// \return Status. + Status Close(); + + /// \brief Checks if the prepared statement is closed. + /// \return The state of the prepared statement. + bool IsClosed() const; + }; }; -} // namespace internal - -using FlightSqlClient = internal::FlightSqlClientT<>; -using PreparedStatement = internal::PreparedStatementT<>; - } // namespace sql } // namespace flight } // namespace arrow - -#include diff --git a/cpp/src/arrow/flight/flight-sql/client_internal.cc b/cpp/src/arrow/flight/flight-sql/client_internal.cc new file mode 100644 index 00000000000..93e79f64fa7 --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/client_internal.cc @@ -0,0 +1,94 @@ +// 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 + +namespace arrow { +namespace flight { +namespace sql { +namespace internal { +class FlightClientImpl { + public: + explicit FlightClientImpl(std::unique_ptr client) + : client_(std::move(client)) {} + + ~FlightClientImpl() = default; + + Status GetFlightInfo(const FlightCallOptions& options, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { + return client_->GetFlightInfo(options, descriptor, info); + } + + Status DoPut(const FlightCallOptions& options, const FlightDescriptor& descriptor, + const std::shared_ptr& schema, + std::unique_ptr* stream, + std::unique_ptr* reader) { + return client_->DoPut(options, descriptor, schema, stream, reader); + } + + Status DoGet(const FlightCallOptions& options, const Ticket& ticket, + std::unique_ptr* stream) { + return client_->DoGet(options, ticket, stream); + } + + Status DoAction(const FlightCallOptions& options, const Action& action, + std::unique_ptr* results) { + return client_->DoAction(options, action, results); + } + + private: + std::unique_ptr client_; +}; + +Status FlightClientImpl_GetFlightInfo(FlightClientImpl& client, + const FlightCallOptions& options, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { + return client.GetFlightInfo(options, descriptor, info); +} + +Status FlightClientImpl_DoPut(FlightClientImpl& client, const FlightCallOptions& options, + const FlightDescriptor& descriptor, + const std::shared_ptr& schema, + std::unique_ptr* stream, + std::unique_ptr* reader) { + return client.DoPut(options, descriptor, schema, stream, reader); +} + +Status FlightClientImpl_DoGet(FlightClientImpl& client, const FlightCallOptions& options, + const Ticket& ticket, + std::unique_ptr* stream) { + return client.DoGet(options, ticket, stream); +} + +Status FlightClientImpl_DoAction(FlightClientImpl& client, + const FlightCallOptions& options, const Action& action, + std::unique_ptr* results) { + return client.DoAction(options, action, results); +} + +std::shared_ptr FlightClientImpl_Create( + std::unique_ptr client) { + return std::make_shared(std::move(client)); +} + +} // namespace internal +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/client_internal.h b/cpp/src/arrow/flight/flight-sql/client_internal.h new file mode 100644 index 00000000000..12ac4cd6e1d --- /dev/null +++ b/cpp/src/arrow/flight/flight-sql/client_internal.h @@ -0,0 +1,59 @@ +// 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 + +namespace arrow { +namespace flight { + +class FlightClient; +class FlightCallOptions; +class FlightStreamWriter; +class FlightMetadataReader; +class FlightStreamReader; + +namespace sql { +namespace internal { + +class FlightClientImpl; + +Status FlightClientImpl_GetFlightInfo(FlightClientImpl& client, + const FlightCallOptions& options, + const FlightDescriptor& descriptor, + std::unique_ptr* info); + +Status FlightClientImpl_DoPut(FlightClientImpl& client, const FlightCallOptions& options, + const FlightDescriptor& descriptor, + const std::shared_ptr& schema, + std::unique_ptr* stream, + std::unique_ptr* reader); + +Status FlightClientImpl_DoGet(FlightClientImpl& client, const FlightCallOptions& options, + const Ticket& ticket, + std::unique_ptr* stream); + +Status FlightClientImpl_DoAction(FlightClientImpl& client, + const FlightCallOptions& options, const Action& action, + std::unique_ptr* results); + +std::shared_ptr FlightClientImpl_Create( + std::unique_ptr client); + +} // namespace internal +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight-sql/client_test.cc index 77376f704c1..405e1b6667b 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight-sql/client_test.cc @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. -#include +#include #include #include -#include +#include #include #include #include @@ -33,13 +33,17 @@ namespace arrow { namespace flight { namespace sql { -using internal::FlightSqlClientT; - -class FlightClientMock { +namespace internal { +class FlightClientImpl { public: + ~FlightClientImpl() = default; + MOCK_METHOD(Status, GetFlightInfo, (const FlightCallOptions&, const FlightDescriptor&, std::unique_ptr*)); + 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, @@ -50,6 +54,35 @@ class FlightClientMock { std::unique_ptr* results)); }; +Status FlightClientImpl_GetFlightInfo(FlightClientImpl* client, + const FlightCallOptions& options, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { + return client->GetFlightInfo(options, descriptor, info); +} + +Status FlightClientImpl_DoPut(FlightClientImpl* client, const FlightCallOptions& options, + const FlightDescriptor& descriptor, + const std::shared_ptr& schema, + std::unique_ptr* stream, + std::unique_ptr* reader) { + return client->DoPut(options, descriptor, schema, stream, reader); +} + +Status FlightClientImpl_DoGet(FlightClientImpl* client, const FlightCallOptions& options, + const Ticket& ticket, + std::unique_ptr* stream) { + return client->DoGet(options, ticket, stream); +} + +Status FlightClientImpl_DoAction(FlightClientImpl* client, + const FlightCallOptions& options, const Action& action, + std::unique_ptr* results) { + return client->DoAction(options, action, results); +} + +} // namespace internal + class FlightMetadataReaderMock : public FlightMetadataReader { public: std::shared_ptr* buffer; @@ -104,9 +137,8 @@ FlightDescriptor getDescriptor(google::protobuf::Message& command) { } TEST(TestFlightSqlClient, TestGetCatalogs) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); FlightCallOptions call_options; pb::sql::CommandGetCatalogs command; @@ -118,9 +150,8 @@ TEST(TestFlightSqlClient, TestGetCatalogs) { } TEST(TestFlightSqlClient, TestGetSchemas) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); FlightCallOptions call_options; std::string schema_filter_pattern = "schema_filter_pattern"; @@ -137,9 +168,8 @@ TEST(TestFlightSqlClient, TestGetSchemas) { } TEST(TestFlightSqlClient, TestGetTables) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); FlightCallOptions call_options; std::string catalog = "catalog"; @@ -166,9 +196,8 @@ TEST(TestFlightSqlClient, TestGetTables) { } TEST(TestFlightSqlClient, TestGetTableTypes) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); FlightCallOptions call_options; pb::sql::CommandGetTableTypes command; @@ -180,9 +209,8 @@ TEST(TestFlightSqlClient, TestGetTableTypes) { } TEST(TestFlightSqlClient, TestGetExported) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); FlightCallOptions call_options; std::string catalog = "catalog"; @@ -201,9 +229,8 @@ TEST(TestFlightSqlClient, TestGetExported) { } TEST(TestFlightSqlClient, TestGetImported) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); FlightCallOptions call_options; std::string catalog = "catalog"; @@ -222,9 +249,8 @@ TEST(TestFlightSqlClient, TestGetImported) { } TEST(TestFlightSqlClient, TestGetPrimary) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); FlightCallOptions call_options; std::string catalog = "catalog"; @@ -243,9 +269,8 @@ TEST(TestFlightSqlClient, TestGetPrimary) { } TEST(TestFlightSqlClient, TestGetCrossReference) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); FlightCallOptions call_options; std::string pk_catalog = "pk_catalog"; @@ -271,9 +296,8 @@ TEST(TestFlightSqlClient, TestGetCrossReference) { } TEST(TestFlightSqlClient, TestExecute) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); FlightCallOptions call_options; std::string query = "query"; @@ -288,9 +312,8 @@ TEST(TestFlightSqlClient, TestExecute) { } TEST(TestFlightSqlClient, TestPreparedStatementExecute) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); FlightCallOptions call_options; const std::string query = "query"; @@ -322,9 +345,8 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecute) { } TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); FlightCallOptions call_options; const std::string query = "query"; @@ -390,9 +412,8 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { } TEST(TestFlightSqlClient, TestExecuteUpdate) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); FlightCallOptions call_options; std::string query = "query"; @@ -435,9 +456,8 @@ TEST(TestFlightSqlClient, TestExecuteUpdate) { } TEST(TestFlightSqlClient, TestGetSqlInfo) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); std::vector sql_info{ pb::sql::SqlInfo::FLIGHT_SQL_SERVER_NAME, @@ -458,9 +478,8 @@ TEST(TestFlightSqlClient, TestGetSqlInfo) { template inline void AssertTestPreparedStatementExecuteUpdateOk( Func func, const std::shared_ptr* schema) { - auto* client_mock = new FlightClientMock(); - std::unique_ptr client_mock_ptr(client_mock); - FlightSqlClientT sql_client(client_mock_ptr); + auto client_mock = std::make_shared(); + FlightSqlClient sql_client(client_mock); const std::string query = "SELECT * FROM IRRELEVANT"; const FlightCallOptions call_options; @@ -519,9 +538,8 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateNoParameterBinding) { AssertTestPreparedStatementExecuteUpdateOk( - [](const std::shared_ptr>& - prepared_statement, - FlightClientMock& client_mock, const std::shared_ptr* schema, + [](const std::shared_ptr& prepared_statement, + internal::FlightClientImpl& client_mock, const std::shared_ptr* schema, const int64_t& row_count) {}, NULLPTR); } @@ -530,9 +548,8 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding const auto schema = arrow::schema( {arrow::field("field0", arrow::utf8()), arrow::field("field1", arrow::uint8())}); AssertTestPreparedStatementExecuteUpdateOk( - [](const std::shared_ptr>& - prepared_statement, - FlightClientMock& client_mock, const std::shared_ptr* schema, + [](const std::shared_ptr& prepared_statement, + internal::FlightClientImpl& client_mock, const std::shared_ptr* schema, const int64_t& row_count) { std::shared_ptr string_array; std::shared_ptr uint8_array; diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h index 9b5bb0b5d02..e0dca0260b4 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h @@ -21,6 +21,7 @@ #include #include +#include #include "arrow/api.h" #include "arrow/flight/flight-sql/example/sqlite_statement.h" diff --git a/cpp/src/arrow/flight/flight-sql/server.cc b/cpp/src/arrow/flight/flight-sql/server.cc index 469f74d9456..e520e76ac24 100644 --- a/cpp/src/arrow/flight/flight-sql/server.cc +++ b/cpp/src/arrow/flight/flight-sql/server.cc @@ -20,6 +20,9 @@ #include "arrow/flight/flight-sql/server.h" +#include +#include + #include #include #include diff --git a/cpp/src/arrow/flight/flight-sql/server.h b/cpp/src/arrow/flight/flight-sql/server.h index f754fdde0da..f02d454136c 100644 --- a/cpp/src/arrow/flight/flight-sql/server.h +++ b/cpp/src/arrow/flight/flight-sql/server.h @@ -20,9 +20,7 @@ #pragma once -#include #include -#include #include #include diff --git a/cpp/src/arrow/flight/flight-sql/server_test.cc b/cpp/src/arrow/flight/flight-sql/server_test.cc index 0ceff978c51..5063cf31dda 100644 --- a/cpp/src/arrow/flight/flight-sql/server_test.cc +++ b/cpp/src/arrow/flight/flight-sql/server_test.cc @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -55,7 +56,7 @@ class TestFlightSqlServer : public ::testing::Environment { ASSERT_OK(Location::Parse(uri, &location)); ASSERT_OK(FlightClient::Connect(location, &client)); - sql_client = new FlightSqlClient(client); + sql_client = new FlightSqlClient(std::move(client)); } void TearDown() override { diff --git a/cpp/src/arrow/flight/flight-sql/test_app_cli.cc b/cpp/src/arrow/flight/flight-sql/test_app_cli.cc index eb035e6fc21..c195ea2d6a1 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app_cli.cc @@ -16,6 +16,7 @@ // under the License. #include +#include #include #include #include @@ -113,7 +114,7 @@ Status RunMain() { call_options.headers.push_back(bearer_result.ValueOrDie()); } - FlightSqlClient sqlClient(client); + FlightSqlClient sqlClient(std::move(client)); if (fLS::FLAGS_command == "ExecuteUpdate") { ARROW_ASSIGN_OR_RAISE(auto rows, diff --git a/cpp/src/arrow/flight/flight-sql/test_server_cli.cc b/cpp/src/arrow/flight/flight-sql/test_server_cli.cc index 9e89f332670..45565115331 100644 --- a/cpp/src/arrow/flight/flight-sql/test_server_cli.cc +++ b/cpp/src/arrow/flight/flight-sql/test_server_cli.cc @@ -31,8 +31,6 @@ DEFINE_int32(port, 31337, "Server port to listen on"); -namespace pb = arrow::flight::protocol; - std::unique_ptr g_server; int main(int argc, char** argv) { From 838469fae0a8cab867d3574a74d0ffccd6a5faa4 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 28 Oct 2021 17:23:04 -0300 Subject: [PATCH 168/237] Remove explicit using of GFlags namespaces --- cpp/src/arrow/flight/flight-sql/server.h | 8 +-- .../arrow/flight/flight-sql/test_app_cli.cc | 63 +++++++++---------- 2 files changed, 33 insertions(+), 38 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/server.h b/cpp/src/arrow/flight/flight-sql/server.h index f02d454136c..c0dba93f0b1 100644 --- a/cpp/src/arrow/flight/flight-sql/server.h +++ b/cpp/src/arrow/flight/flight-sql/server.h @@ -139,13 +139,13 @@ class FlightSqlServerBase : public FlightServerBase { const ActionType FLIGHT_SQL_CREATE_PREPARED_STATEMENT = ActionType{"CreatePreparedStatement", "Creates a reusable prepared statement resource on the server.\n" - "Request Message: ActionCreatePreparedStatementRequest\n" - "Response Message: ActionCreatePreparedStatementResult"}; + "Request Message: ActionCreatePreparedStatementRequest\n" + "Response Message: ActionCreatePreparedStatementResult"}; const ActionType FLIGHT_SQL_CLOSE_PREPARED_STATEMENT = ActionType{"ClosePreparedStatement", "Closes a reusable prepared statement resource on the server.\n" - "Request Message: ActionClosePreparedStatementRequest\n" - "Response Message: N/A"}; + "Request Message: ActionClosePreparedStatementRequest\n" + "Response Message: N/A"}; Status ListActions(const ServerCallContext& context, std::vector* actions) override; diff --git a/cpp/src/arrow/flight/flight-sql/test_app_cli.cc b/cpp/src/arrow/flight/flight-sql/test_app_cli.cc index c195ea2d6a1..d2b075c4d88 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app_cli.cc @@ -101,14 +101,14 @@ Status PrintResults(FlightSqlClient& client, const FlightCallOptions& call_optio Status RunMain() { std::unique_ptr client; Location location; - ARROW_RETURN_NOT_OK(Location::ForGrpcTcp(fLS::FLAGS_host, fLI::FLAGS_port, &location)); + ARROW_RETURN_NOT_OK(Location::ForGrpcTcp(FLAGS_host, FLAGS_port, &location)); ARROW_RETURN_NOT_OK(FlightClient::Connect(location, &client)); FlightCallOptions call_options; - if (!fLS::FLAGS_username.empty() || !fLS::FLAGS_password.empty()) { + if (!FLAGS_username.empty() || !FLAGS_password.empty()) { Result> bearer_result = - client->AuthenticateBasicToken({}, fLS::FLAGS_username, fLS::FLAGS_password); + client->AuthenticateBasicToken({}, FLAGS_username, FLAGS_password); ARROW_RETURN_NOT_OK(bearer_result); call_options.headers.push_back(bearer_result.ValueOrDie()); @@ -116,9 +116,8 @@ Status RunMain() { FlightSqlClient sqlClient(std::move(client)); - if (fLS::FLAGS_command == "ExecuteUpdate") { - ARROW_ASSIGN_OR_RAISE(auto rows, - sqlClient.ExecuteUpdate(call_options, fLS::FLAGS_query)); + if (FLAGS_command == "ExecuteUpdate") { + ARROW_ASSIGN_OR_RAISE(auto rows, sqlClient.ExecuteUpdate(call_options, FLAGS_query)); std::cout << "Result: " << rows << std::endl; @@ -127,17 +126,16 @@ Status RunMain() { std::unique_ptr info; - if (fLS::FLAGS_command == "Execute") { - ARROW_ASSIGN_OR_RAISE(info, sqlClient.Execute(call_options, fLS::FLAGS_query)); - } else if (fLS::FLAGS_command == "GetCatalogs") { + if (FLAGS_command == "Execute") { + ARROW_ASSIGN_OR_RAISE(info, sqlClient.Execute(call_options, FLAGS_query)); + } else if (FLAGS_command == "GetCatalogs") { ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetCatalogs(call_options)); - } else if (fLS::FLAGS_command == "PreparedStatementExecute") { + } else if (FLAGS_command == "PreparedStatementExecute") { ARROW_ASSIGN_OR_RAISE(auto prepared_statement, - sqlClient.Prepare(call_options, fLS::FLAGS_query)); + sqlClient.Prepare(call_options, FLAGS_query)); ARROW_ASSIGN_OR_RAISE(info, prepared_statement->Execute()); - } else if (fLS::FLAGS_command == "PreparedStatementExecuteParameterBinding") { - ARROW_ASSIGN_OR_RAISE(auto prepared_statement, - sqlClient.Prepare({}, fLS::FLAGS_query)); + } else if (FLAGS_command == "PreparedStatementExecuteParameterBinding") { + ARROW_ASSIGN_OR_RAISE(auto prepared_statement, sqlClient.Prepare({}, FLAGS_query)); ARROW_ASSIGN_OR_RAISE(auto parameter_schema, prepared_statement->GetParameterSchema()); ARROW_ASSIGN_OR_RAISE(auto result_set_schema, @@ -153,34 +151,31 @@ Status RunMain() { ARROW_RETURN_NOT_OK(prepared_statement->SetParameters(result)); ARROW_ASSIGN_OR_RAISE(info, prepared_statement->Execute()); - } else if (fLS::FLAGS_command == "GetSchemas") { - ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetSchemas(call_options, &fLS::FLAGS_catalog, - &fLS::FLAGS_schema)); - } else if (fLS::FLAGS_command == "GetTableTypes") { + } else if (FLAGS_command == "GetSchemas") { + ARROW_ASSIGN_OR_RAISE( + info, sqlClient.GetSchemas(call_options, &FLAGS_catalog, &FLAGS_schema)); + } else if (FLAGS_command == "GetTableTypes") { ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetTableTypes(call_options)); - } else if (fLS::FLAGS_command == "GetTables") { + } else if (FLAGS_command == "GetTables") { std::vector table_types = {}; bool include_schema = false; - ARROW_ASSIGN_OR_RAISE( - info, sqlClient.GetTables(call_options, &fLS::FLAGS_catalog, &fLS::FLAGS_schema, - &fLS::FLAGS_table, include_schema, table_types)); - } else if (fLS::FLAGS_command == "GetExportedKeys") { - ARROW_ASSIGN_OR_RAISE( - info, sqlClient.GetExportedKeys(call_options, &fLS::FLAGS_catalog, - &fLS::FLAGS_schema, fLS::FLAGS_table)); - } else if (fLS::FLAGS_command == "GetImportedKeys") { - ARROW_ASSIGN_OR_RAISE( - info, sqlClient.GetImportedKeys(call_options, &fLS::FLAGS_catalog, - &fLS::FLAGS_schema, fLS::FLAGS_table)); - } else if (fLS::FLAGS_command == "GetPrimaryKeys") { ARROW_ASSIGN_OR_RAISE(info, - sqlClient.GetPrimaryKeys(call_options, &fLS::FLAGS_catalog, - &fLS::FLAGS_schema, fLS::FLAGS_table)); + sqlClient.GetTables(call_options, &FLAGS_catalog, &FLAGS_schema, + &FLAGS_table, include_schema, table_types)); + } else if (FLAGS_command == "GetExportedKeys") { + ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetExportedKeys(call_options, &FLAGS_catalog, + &FLAGS_schema, FLAGS_table)); + } else if (FLAGS_command == "GetImportedKeys") { + ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetImportedKeys(call_options, &FLAGS_catalog, + &FLAGS_schema, FLAGS_table)); + } else if (FLAGS_command == "GetPrimaryKeys") { + ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetPrimaryKeys(call_options, &FLAGS_catalog, + &FLAGS_schema, FLAGS_table)); } if (info != NULLPTR && - !boost::istarts_with(fLS::FLAGS_command, "PreparedStatementExecute")) { + !boost::istarts_with(FLAGS_command, "PreparedStatementExecute")) { return PrintResults(sqlClient, call_options, info); } From d89a82ee66a67e22e88dca5a766a3c0e57d9c1f1 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 28 Oct 2021 17:57:08 -0300 Subject: [PATCH 169/237] Remove unnecessary const modifiers on client interface --- cpp/src/arrow/flight/flight-sql/client.cc | 29 +++++++++-------- cpp/src/arrow/flight/flight-sql/client.h | 31 +++++++++---------- .../arrow/flight/flight-sql/test_app_cli.cc | 2 +- 3 files changed, 30 insertions(+), 32 deletions(-) diff --git a/cpp/src/arrow/flight/flight-sql/client.cc b/cpp/src/arrow/flight/flight-sql/client.cc index ed7e7f4defa..0e311185f57 100644 --- a/cpp/src/arrow/flight/flight-sql/client.cc +++ b/cpp/src/arrow/flight/flight-sql/client.cc @@ -80,7 +80,7 @@ arrow::Result> GetFlightInfoForCommand( } arrow::Result> FlightSqlClient::Execute( - const FlightCallOptions& options, const std::string& query) const { + const FlightCallOptions& options, const std::string& query) { pb::sql::CommandStatementQuery command; command.set_query(query); @@ -88,7 +88,7 @@ arrow::Result> FlightSqlClient::Execute( } arrow::Result FlightSqlClient::ExecuteUpdate(const FlightCallOptions& options, - const std::string& query) const { + const std::string& query) { pb::sql::CommandStatementUpdate command; command.set_query(query); @@ -115,7 +115,7 @@ arrow::Result FlightSqlClient::ExecuteUpdate(const FlightCallOptions& o } arrow::Result> FlightSqlClient::GetCatalogs( - const FlightCallOptions& options) const { + const FlightCallOptions& options) { pb::sql::CommandGetCatalogs command; return GetFlightInfoForCommand(*impl_, options, command); @@ -123,7 +123,7 @@ arrow::Result> FlightSqlClient::GetCatalogs( arrow::Result> FlightSqlClient::GetSchemas( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema_filter_pattern) const { + const std::string* schema_filter_pattern) { pb::sql::CommandGetSchemas command; if (catalog != NULLPTR) { command.set_catalog(*catalog); @@ -138,7 +138,7 @@ arrow::Result> FlightSqlClient::GetSchemas( arrow::Result> FlightSqlClient::GetTables( const FlightCallOptions& options, const std::string* catalog, const std::string* schema_filter_pattern, const std::string* table_filter_pattern, - bool include_schema, std::vector& table_types) const { + bool include_schema, std::vector& table_types) { pb::sql::CommandGetTables command; if (catalog != NULLPTR) { @@ -164,7 +164,7 @@ arrow::Result> FlightSqlClient::GetTables( arrow::Result> FlightSqlClient::GetPrimaryKeys( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table) const { + const std::string* schema, const std::string& table) { pb::sql::CommandGetPrimaryKeys command; if (catalog != NULLPTR) { @@ -182,7 +182,7 @@ arrow::Result> FlightSqlClient::GetPrimaryKeys( arrow::Result> FlightSqlClient::GetExportedKeys( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table) const { + const std::string* schema, const std::string& table) { pb::sql::CommandGetExportedKeys command; if (catalog != NULLPTR) { @@ -200,7 +200,7 @@ arrow::Result> FlightSqlClient::GetExportedKeys( arrow::Result> FlightSqlClient::GetImportedKeys( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table) const { + const std::string* schema, const std::string& table) { pb::sql::CommandGetImportedKeys command; if (catalog != NULLPTR) { @@ -220,7 +220,7 @@ arrow::Result> FlightSqlClient::GetCrossReference( const FlightCallOptions& options, const std::string* pk_catalog, const std::string* pk_schema, const std::string& pk_table, const std::string* fk_catalog, const std::string* fk_schema, - const std::string& fk_table) const { + const std::string& fk_table) { pb::sql::CommandGetCrossReference command; if (pk_catalog != NULLPTR) { @@ -243,14 +243,14 @@ arrow::Result> FlightSqlClient::GetCrossReference( } arrow::Result> FlightSqlClient::GetTableTypes( - const FlightCallOptions& options) const { + const FlightCallOptions& options) { pb::sql::CommandGetTableTypes command; return GetFlightInfoForCommand(*impl_, options, command); } arrow::Result> FlightSqlClient::DoGet( - const FlightCallOptions& options, const Ticket& ticket) const { + const FlightCallOptions& options, const Ticket& ticket) { std::unique_ptr stream; ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoGet(*impl_, options, ticket, &stream)); @@ -375,7 +375,7 @@ Status FlightSqlClient::PreparedStatement::SetParameters( return Status::OK(); } -bool FlightSqlClient::PreparedStatement::IsClosed() const { return is_closed_; } +bool FlightSqlClient::PreparedStatement::IsClosed() { return is_closed_; } arrow::Result> FlightSqlClient::PreparedStatement::GetResultSetSchema() { @@ -424,7 +424,7 @@ Status FlightSqlClient::PreparedStatement::Close() { } arrow::Result> FlightSqlClient::GetSqlInfo( - const FlightCallOptions& options, const std::vector& sql_info) const { + const FlightCallOptions& options, const std::vector& sql_info) { pb::sql::CommandGetSqlInfo command; for (const int& info : sql_info) command.add_info(info); @@ -432,8 +432,7 @@ arrow::Result> FlightSqlClient::GetSqlInfo( } arrow::Result> FlightSqlClient::GetSqlInfo( - const FlightCallOptions& options, - const std::vector& sql_info) const { + const FlightCallOptions& options, const std::vector& sql_info) { return GetSqlInfo(options, reinterpret_cast&>(sql_info)); } diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight-sql/client.h index 0cba9714406..5e446d77850 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight-sql/client.h @@ -52,20 +52,20 @@ class ARROW_EXPORT FlightSqlClient { /// \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) const; + 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) const; + 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) const; + const FlightCallOptions& options); /// \brief Request a list of schemas. /// \param[in] options RPC-layer hints for this call. @@ -74,7 +74,7 @@ class ARROW_EXPORT FlightSqlClient { /// \return The FlightInfo describing where to access the dataset. arrow::Result> GetSchemas( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema_filter_pattern) const; + const std::string* schema_filter_pattern); /// \brief Given a flight ticket and schema, request to be sent the /// stream. Returns record batch stream reader @@ -82,7 +82,7 @@ class ARROW_EXPORT FlightSqlClient { /// \param[in] ticket The flight ticket to use /// \return The returned RecordBatchReader arrow::Result> DoGet( - const FlightCallOptions& options, const Ticket& ticket) const; + const FlightCallOptions& options, const Ticket& ticket); /// \brief Request a list of tables. /// \param[in] options RPC-layer hints for this call. @@ -96,7 +96,7 @@ class ARROW_EXPORT FlightSqlClient { arrow::Result> GetTables( const FlightCallOptions& options, const std::string* catalog, const std::string* schema_filter_pattern, const std::string* table_filter_pattern, - bool include_schema, std::vector& table_types) const; + bool include_schema, std::vector& table_types); /// \brief Request the primary keys for a table. /// \param[in] options RPC-layer hints for this call. @@ -106,7 +106,7 @@ class ARROW_EXPORT FlightSqlClient { /// \return The FlightInfo describing where to access the dataset. arrow::Result> GetPrimaryKeys( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table) const; + const std::string* schema, const std::string& table); /// \brief Retrieves a description about the foreign key columns that reference the /// primary key columns of the given table. @@ -117,7 +117,7 @@ class ARROW_EXPORT FlightSqlClient { /// \return The FlightInfo describing where to access the dataset. arrow::Result> GetExportedKeys( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table) const; + const std::string* schema, const std::string& table); /// \brief Retrieves the foreign key columns for the given table. /// \param[in] options RPC-layer hints for this call. @@ -127,7 +127,7 @@ class ARROW_EXPORT FlightSqlClient { /// \return The FlightInfo describing where to access the dataset. arrow::Result> GetImportedKeys( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table) const; + const std::string* schema, const std::string& table); /// \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 @@ -144,28 +144,27 @@ class ARROW_EXPORT FlightSqlClient { const FlightCallOptions& options, const std::string* pk_catalog, const std::string* pk_schema, const std::string& pk_table, const std::string* fk_catalog, const std::string* fk_schema, - const std::string& fk_table) const; + const std::string& fk_table); /// \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) const; + 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) const; + arrow::Result> GetSqlInfo(const FlightCallOptions& options, + const std::vector& sql_info); /// \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) const; + const FlightCallOptions& options, const std::vector& sql_info); /// \brief Create a prepared statement object. /// \param[in] options RPC-layer hints for this call. @@ -224,7 +223,7 @@ class ARROW_EXPORT FlightSqlClient { /// \brief Checks if the prepared statement is closed. /// \return The state of the prepared statement. - bool IsClosed() const; + bool IsClosed(); }; }; diff --git a/cpp/src/arrow/flight/flight-sql/test_app_cli.cc b/cpp/src/arrow/flight/flight-sql/test_app_cli.cc index d2b075c4d88..5a7dbe81e6e 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/flight-sql/test_app_cli.cc @@ -55,7 +55,7 @@ DEFINE_string(catalog, "", "Catalog"); DEFINE_string(schema, "", "Schema"); DEFINE_string(table, "", "Table"); -Status PrintResultsForEndpoint(const FlightSqlClient& client, +Status PrintResultsForEndpoint(FlightSqlClient& client, const FlightCallOptions& call_options, const FlightEndpoint& endpoint) { ARROW_ASSIGN_OR_RAISE(auto stream, client.DoGet(call_options, endpoint.ticket)); From 166bb6b40544ea0a58354b1e553711338ceddcea Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 1 Nov 2021 14:02:01 -0300 Subject: [PATCH 170/237] Rename directory flight-sql to flight_sql --- cpp/src/arrow/CMakeLists.txt | 2 +- .../ArrowFlightSqlConfig.cmake.in | 0 .../flight/{flight-sql => flight_sql}/CMakeLists.txt | 4 ++-- cpp/src/arrow/flight/{flight-sql => flight_sql}/api.h | 2 +- .../{flight-sql => flight_sql}/arrow-flight-sql.pc.in | 0 .../arrow/flight/{flight-sql => flight_sql}/client.cc | 6 +++--- .../arrow/flight/{flight-sql => flight_sql}/client.h | 2 +- .../{flight-sql => flight_sql}/client_internal.cc | 2 +- .../{flight-sql => flight_sql}/client_internal.h | 0 .../flight/{flight-sql => flight_sql}/client_test.cc | 4 ++-- .../example/sqlite_server.cc | 10 +++++----- .../{flight-sql => flight_sql}/example/sqlite_server.h | 6 +++--- .../example/sqlite_statement.cc | 4 ++-- .../example/sqlite_statement.h | 0 .../example/sqlite_statement_batch_reader.cc | 4 ++-- .../example/sqlite_statement_batch_reader.h | 2 +- .../example/sqlite_tables_schema_batch_reader.cpp | 8 ++++---- .../example/sqlite_tables_schema_batch_reader.h | 4 ++-- .../arrow/flight/{flight-sql => flight_sql}/server.cc | 4 ++-- .../arrow/flight/{flight-sql => flight_sql}/server.h | 6 +++--- .../flight/{flight-sql => flight_sql}/server_test.cc | 8 ++++---- .../flight/{flight-sql => flight_sql}/test_app_cli.cc | 2 +- .../{flight-sql => flight_sql}/test_server_cli.cc | 2 +- 23 files changed, 41 insertions(+), 41 deletions(-) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/ArrowFlightSqlConfig.cmake.in (100%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/CMakeLists.txt (97%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/api.h (94%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/arrow-flight-sql.pc.in (100%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/client.cc (99%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/client.h (99%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/client_internal.cc (98%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/client_internal.h (100%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/client_test.cc (99%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/example/sqlite_server.cc (98%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/example/sqlite_server.h (97%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/example/sqlite_statement.cc (97%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/example/sqlite_statement.h (100%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/example/sqlite_statement_batch_reader.cc (98%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/example/sqlite_statement_batch_reader.h (97%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/example/sqlite_tables_schema_batch_reader.cpp (93%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/example/sqlite_tables_schema_batch_reader.h (93%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/server.cc (99%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/server.h (99%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/server_test.cc (99%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/test_app_cli.cc (99%) rename cpp/src/arrow/flight/{flight-sql => flight_sql}/test_server_cli.cc (97%) diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 747bfaa489f..01b646d4e76 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -733,7 +733,7 @@ if(ARROW_FLIGHT) endif() if(ARROW_FLIGHT_SQL) - add_subdirectory(flight/flight-sql) + add_subdirectory(flight/flight_sql) endif() if(ARROW_HIVESERVER2) diff --git a/cpp/src/arrow/flight/flight-sql/ArrowFlightSqlConfig.cmake.in b/cpp/src/arrow/flight/flight_sql/ArrowFlightSqlConfig.cmake.in similarity index 100% rename from cpp/src/arrow/flight/flight-sql/ArrowFlightSqlConfig.cmake.in rename to cpp/src/arrow/flight/flight_sql/ArrowFlightSqlConfig.cmake.in diff --git a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt similarity index 97% rename from cpp/src/arrow/flight/flight-sql/CMakeLists.txt rename to cpp/src/arrow/flight/flight_sql/CMakeLists.txt index ce634f1513b..d9662817255 100644 --- a/cpp/src/arrow/flight/flight-sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -17,7 +17,7 @@ add_custom_target(arrow_flight_sql) -arrow_install_all_headers("arrow/flight/flight-sql") +arrow_install_all_headers("arrow/flight/flight_sql") set(FLIGHT_SQL_PROTO_PATH "${ARROW_SOURCE_DIR}/../format") set(FLIGHT_SQL_PROTO ${ARROW_SOURCE_DIR}/../format/FlightSql.proto) @@ -48,7 +48,7 @@ add_arrow_lib(arrow_flight_sql SOURCES ${ARROW_FLIGHT_SQL_SRCS} PRECOMPILED_HEADERS - "$<$:arrow/flight/flight-sql/pch.h>" + "$<$:arrow/flight/flight_sql/pch.h>" DEPENDENCIES flight_sql_protobuf_gen SHARED_LINK_FLAGS diff --git a/cpp/src/arrow/flight/flight-sql/api.h b/cpp/src/arrow/flight/flight_sql/api.h similarity index 94% rename from cpp/src/arrow/flight/flight-sql/api.h rename to cpp/src/arrow/flight/flight_sql/api.h index 7c2d7f6ead3..8d5124ee86d 100644 --- a/cpp/src/arrow/flight/flight-sql/api.h +++ b/cpp/src/arrow/flight/flight_sql/api.h @@ -17,4 +17,4 @@ #pragma once -#include "arrow/flight/flight-sql/client.h" +#include "arrow/flight/flight_sql/client.h" diff --git a/cpp/src/arrow/flight/flight-sql/arrow-flight-sql.pc.in b/cpp/src/arrow/flight/flight_sql/arrow-flight-sql.pc.in similarity index 100% rename from cpp/src/arrow/flight/flight-sql/arrow-flight-sql.pc.in rename to cpp/src/arrow/flight/flight_sql/arrow-flight-sql.pc.in diff --git a/cpp/src/arrow/flight/flight-sql/client.cc b/cpp/src/arrow/flight/flight_sql/client.cc similarity index 99% rename from cpp/src/arrow/flight/flight-sql/client.cc rename to cpp/src/arrow/flight/flight_sql/client.cc index 0e311185f57..e9608331129 100644 --- a/cpp/src/arrow/flight/flight-sql/client.cc +++ b/cpp/src/arrow/flight/flight_sql/client.cc @@ -16,9 +16,9 @@ // under the License. #include -#include -#include -#include +#include +#include +#include #include #include #include diff --git a/cpp/src/arrow/flight/flight-sql/client.h b/cpp/src/arrow/flight/flight_sql/client.h similarity index 99% rename from cpp/src/arrow/flight/flight-sql/client.h rename to cpp/src/arrow/flight/flight_sql/client.h index 5e446d77850..b1ddf5ddef8 100644 --- a/cpp/src/arrow/flight/flight-sql/client.h +++ b/cpp/src/arrow/flight/flight_sql/client.h @@ -19,7 +19,7 @@ #include #include -#include +#include #include #include #include diff --git a/cpp/src/arrow/flight/flight-sql/client_internal.cc b/cpp/src/arrow/flight/flight_sql/client_internal.cc similarity index 98% rename from cpp/src/arrow/flight/flight-sql/client_internal.cc rename to cpp/src/arrow/flight/flight_sql/client_internal.cc index 93e79f64fa7..dcb645c6875 100644 --- a/cpp/src/arrow/flight/flight-sql/client_internal.cc +++ b/cpp/src/arrow/flight/flight_sql/client_internal.cc @@ -16,7 +16,7 @@ // under the License. #include -#include +#include namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight-sql/client_internal.h b/cpp/src/arrow/flight/flight_sql/client_internal.h similarity index 100% rename from cpp/src/arrow/flight/flight-sql/client_internal.h rename to cpp/src/arrow/flight/flight_sql/client_internal.h diff --git a/cpp/src/arrow/flight/flight-sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc similarity index 99% rename from cpp/src/arrow/flight/flight-sql/client_test.cc rename to cpp/src/arrow/flight/flight_sql/client_test.cc index 405e1b6667b..2e9886b291f 100644 --- a/cpp/src/arrow/flight/flight-sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -16,8 +16,8 @@ // under the License. #include -#include -#include +#include +#include #include #include #include diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc similarity index 98% rename from cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc rename to cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc index a8d0c82369e..43697ae6f44 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight-sql/example/sqlite_server.h" +#include "arrow/flight/flight_sql/example/sqlite_server.h" #include @@ -26,10 +26,10 @@ #include #include "arrow/api.h" -#include "arrow/flight/flight-sql/example/sqlite_statement.h" -#include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" -#include "arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h" -#include "arrow/flight/flight-sql/server.h" +#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h" +#include "arrow/flight/flight_sql/server.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h similarity index 97% rename from cpp/src/arrow/flight/flight-sql/example/sqlite_server.h rename to cpp/src/arrow/flight/flight_sql/example/sqlite_server.h index e0dca0260b4..8fdcec87d8d 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h @@ -24,9 +24,9 @@ #include #include "arrow/api.h" -#include "arrow/flight/flight-sql/example/sqlite_statement.h" -#include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" -#include "arrow/flight/flight-sql/server.h" +#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/flight_sql/server.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc similarity index 97% rename from cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc rename to cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc index d983ada5aee..723f4323efc 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc @@ -15,14 +15,14 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight-sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/example/sqlite_statement.h" #include #include #include "arrow/api.h" -#include "arrow/flight/flight-sql/example/sqlite_server.h" +#include "arrow/flight/flight_sql/example/sqlite_server.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h similarity index 100% rename from cpp/src/arrow/flight/flight-sql/example/sqlite_statement.h rename to cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc similarity index 98% rename from cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc rename to cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc index 1620622475c..27c3d01816f 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc @@ -15,12 +15,12 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" #include #include "arrow/api.h" -#include "arrow/flight/flight-sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/example/sqlite_statement.h" #define STRING_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ case TYPE_CLASS##Type::type_id: { \ diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h similarity index 97% rename from cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h rename to cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h index e86ac0725b6..2ae8db6e8e6 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h @@ -20,7 +20,7 @@ #include #include "arrow/api.h" -#include "arrow/flight/flight-sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/example/sqlite_statement.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cpp similarity index 93% rename from cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp rename to cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cpp index ddb4e2e6c96..d78684836e4 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.cpp +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cpp @@ -15,17 +15,17 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h" +#include "arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h" -#include +#include #include #include #include #include -#include "arrow/flight/flight-sql/example/sqlite_server.h" -#include "arrow/flight/flight-sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/example/sqlite_server.h" +#include "arrow/flight/flight_sql/example/sqlite_statement.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h similarity index 93% rename from cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h rename to cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h index 10c210bb76f..0fe7cef7efb 100644 --- a/cpp/src/arrow/flight/flight-sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h @@ -23,8 +23,8 @@ #include #include -#include "arrow/flight/flight-sql/example/sqlite_statement.h" -#include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight-sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc similarity index 99% rename from cpp/src/arrow/flight/flight-sql/server.cc rename to cpp/src/arrow/flight/flight_sql/server.cc index e520e76ac24..e0bbaeafdfd 100644 --- a/cpp/src/arrow/flight/flight-sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -18,9 +18,9 @@ // Interfaces to use for defining Flight RPC servers. API should be considered // experimental for now -#include "arrow/flight/flight-sql/server.h" +#include "arrow/flight/flight_sql/server.h" -#include +#include #include #include diff --git a/cpp/src/arrow/flight/flight-sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h similarity index 99% rename from cpp/src/arrow/flight/flight-sql/server.h rename to cpp/src/arrow/flight/flight_sql/server.h index c0dba93f0b1..187a5ba07f8 100644 --- a/cpp/src/arrow/flight/flight-sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -27,9 +27,9 @@ #include #include "arrow/api.h" -#include "arrow/flight/flight-sql/example/sqlite_statement.h" -#include "arrow/flight/flight-sql/example/sqlite_statement_batch_reader.h" -#include "arrow/flight/flight-sql/server.h" +#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/flight_sql/server.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight-sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc similarity index 99% rename from cpp/src/arrow/flight/flight-sql/server_test.cc rename to cpp/src/arrow/flight/flight_sql/server_test.cc index 5063cf31dda..f466b320584 100644 --- a/cpp/src/arrow/flight/flight-sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -17,10 +17,10 @@ #include #include -#include -#include -#include -#include +#include +#include +#include +#include #include #include #include diff --git a/cpp/src/arrow/flight/flight-sql/test_app_cli.cc b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc similarity index 99% rename from cpp/src/arrow/flight/flight-sql/test_app_cli.cc rename to cpp/src/arrow/flight/flight_sql/test_app_cli.cc index 5a7dbe81e6e..c8e3de9fae5 100644 --- a/cpp/src/arrow/flight/flight-sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/cpp/src/arrow/flight/flight-sql/test_server_cli.cc b/cpp/src/arrow/flight/flight_sql/test_server_cli.cc similarity index 97% rename from cpp/src/arrow/flight/flight-sql/test_server_cli.cc rename to cpp/src/arrow/flight/flight_sql/test_server_cli.cc index 45565115331..0e0d63a272d 100644 --- a/cpp/src/arrow/flight/flight-sql/test_server_cli.cc +++ b/cpp/src/arrow/flight/flight_sql/test_server_cli.cc @@ -21,7 +21,7 @@ #include #include -#include "arrow/flight/flight-sql/example/sqlite_server.h" +#include "arrow/flight/flight_sql/example/sqlite_server.h" #include "arrow/flight/server.h" #include "arrow/flight/test_integration.h" #include "arrow/flight/test_util.h" From 8095b6e9b8593fb15d528c863f9d9a8963b02013 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 1 Nov 2021 14:04:48 -0300 Subject: [PATCH 171/237] Remove unused includes on server.h --- cpp/src/arrow/flight/flight_sql/server.h | 4 ---- 1 file changed, 4 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index 187a5ba07f8..79bc293a7e0 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -21,10 +21,6 @@ #pragma once #include -#include - -#include -#include #include "arrow/api.h" #include "arrow/flight/flight_sql/example/sqlite_statement.h" From b26bfc40d4a85d0a685d7a985e1541828f613027 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 2 Nov 2021 14:54:53 -0300 Subject: [PATCH 172/237] [C++] Other fixes for ratification (#190) * Rename sqlClient to sql_client on test_app_cli.cc * Add missing parameters on PreparedStatement constructor * Add NOTE to PreparedStatement destructor * Parse PreparedStatement's dataset and parameters schema when constructing * Rename Flight SQL Actions constants * Remove unnecessary 'using' keyword on server.h * Clean up header files and includes * Rename getters for schemas on PreparedStatement and make them const * Handle possible protobuf parsing errors on server.cc * Move CreateStatementQueryTicket implementation to sql namespace --- cpp/src/arrow/flight/flight_sql/api.h | 2 +- cpp/src/arrow/flight/flight_sql/client.cc | 75 ++++++++++--------- cpp/src/arrow/flight/flight_sql/client.h | 37 +++++---- .../arrow/flight/flight_sql/client_internal.h | 2 + .../arrow/flight/flight_sql/client_test.cc | 9 +-- .../flight_sql/example/sqlite_server.cc | 14 ++-- .../flight/flight_sql/example/sqlite_server.h | 11 +-- .../flight_sql/example/sqlite_statement.cc | 7 +- .../flight_sql/example/sqlite_statement.h | 4 +- .../example/sqlite_statement_batch_reader.cc | 8 +- .../example/sqlite_statement_batch_reader.h | 5 +- .../sqlite_tables_schema_batch_reader.cpp | 6 +- .../sqlite_tables_schema_batch_reader.h | 9 +-- cpp/src/arrow/flight/flight_sql/server.cc | 55 +++++++------- cpp/src/arrow/flight/flight_sql/server.h | 49 ++++++------ .../arrow/flight/flight_sql/server_test.cc | 6 +- .../arrow/flight/flight_sql/test_app_cli.cc | 42 +++++------ .../flight/flight_sql/test_server_cli.cc | 15 ++-- 18 files changed, 179 insertions(+), 177 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/api.h b/cpp/src/arrow/flight/flight_sql/api.h index 8d5124ee86d..ddb83b25b1c 100644 --- a/cpp/src/arrow/flight/flight_sql/api.h +++ b/cpp/src/arrow/flight/flight_sql/api.h @@ -17,4 +17,4 @@ #pragma once -#include "arrow/flight/flight_sql/client.h" +#include diff --git a/cpp/src/arrow/flight/flight_sql/client.cc b/cpp/src/arrow/flight/flight_sql/client.cc index e9608331129..8e0fc34f55d 100644 --- a/cpp/src/arrow/flight/flight_sql/client.cc +++ b/cpp/src/arrow/flight/flight_sql/client.cc @@ -27,8 +27,7 @@ #include #include -#include -#include +namespace pb = arrow::flight::protocol; namespace arrow { namespace flight { @@ -41,17 +40,21 @@ FlightSqlClient::FlightSqlClient(std::unique_ptr client) : impl_(internal::FlightClientImpl_Create(std::move(client))) {} FlightSqlClient::PreparedStatement::PreparedStatement( - std::shared_ptr client, const std::string& query, - pb::sql::ActionCreatePreparedStatementResult& prepared_statement_result, + std::shared_ptr client, std::string handle, + std::shared_ptr dataset_schema, std::shared_ptr parameter_schema, FlightCallOptions options) - : client_(client), + : client_(std::move(client)), options_(std::move(options)), - prepared_statement_result_(std::move(prepared_statement_result)), + handle_(std::move(handle)), + dataset_schema_(std::move(dataset_schema)), + parameter_schema_(std::move(parameter_schema)), is_closed_(false) {} FlightSqlClient::~FlightSqlClient() = default; FlightSqlClient::PreparedStatement::~PreparedStatement() { + if (IsClosed()) return; + const Status status = Close(); if (!status.ok()) { ARROW_LOG(ERROR) << "Failed to delete PreparedStatement: " << status.ToString(); @@ -290,8 +293,28 @@ FlightSqlClient::Prepare(const FlightCallOptions& options, const std::string& qu return Status::Invalid("Unable to unpack ActionCreatePreparedStatementResult"); } - return std::make_shared(impl_, query, prepared_statement_result, - options); + 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(impl_, handle, dataset_schema, + parameter_schema, options); } arrow::Result> FlightSqlClient::PreparedStatement::Execute() { @@ -301,8 +324,7 @@ arrow::Result> FlightSqlClient::PreparedStatement::E pb::sql::CommandPreparedStatementQuery execute_query_command; - execute_query_command.set_prepared_statement_handle( - prepared_statement_result_.prepared_statement_handle()); + execute_query_command.set_prepared_statement_handle(handle_); google::protobuf::Any any; any.PackFrom(execute_query_command); @@ -336,8 +358,7 @@ arrow::Result FlightSqlClient::PreparedStatement::ExecuteUpdate() { } pb::sql::CommandPreparedStatementUpdate command; - command.set_prepared_statement_handle( - prepared_statement_result_.prepared_statement_handle()); + command.set_prepared_statement_handle(handle_); const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); std::unique_ptr writer; std::unique_ptr reader; @@ -377,26 +398,12 @@ Status FlightSqlClient::PreparedStatement::SetParameters( bool FlightSqlClient::PreparedStatement::IsClosed() { return is_closed_; } -arrow::Result> -FlightSqlClient::PreparedStatement::GetResultSetSchema() { - auto& args = prepared_statement_result_.dataset_schema(); - std::shared_ptr schema_buffer = std::make_shared(args); - - io::BufferReader reader(schema_buffer); - - ipc::DictionaryMemo in_memo; - return ReadSchema(&reader, &in_memo); +std::shared_ptr FlightSqlClient::PreparedStatement::dataset_schema() const { + return dataset_schema_; } -arrow::Result> -FlightSqlClient::PreparedStatement::GetParameterSchema() { - auto& args = prepared_statement_result_.parameter_schema(); - std::shared_ptr schema_buffer = std::make_shared(args); - - io::BufferReader reader(schema_buffer); - - ipc::DictionaryMemo in_memo; - return ReadSchema(&reader, &in_memo); +std::shared_ptr FlightSqlClient::PreparedStatement::parameter_schema() const { + return parameter_schema_; } Status FlightSqlClient::PreparedStatement::Close() { @@ -405,8 +412,7 @@ Status FlightSqlClient::PreparedStatement::Close() { } google::protobuf::Any command; pb::sql::ActionClosePreparedStatementRequest request; - request.set_prepared_statement_handle( - prepared_statement_result_.prepared_statement_handle()); + request.set_prepared_statement_handle(handle_); command.PackFrom(request); @@ -431,11 +437,6 @@ arrow::Result> FlightSqlClient::GetSqlInfo( return GetFlightInfoForCommand(*impl_, options, command); } -arrow::Result> FlightSqlClient::GetSqlInfo( - const FlightCallOptions& options, const std::vector& sql_info) { - return GetSqlInfo(options, reinterpret_cast&>(sql_info)); -} - } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight_sql/client.h b/cpp/src/arrow/flight/flight_sql/client.h index b1ddf5ddef8..218e949ed89 100644 --- a/cpp/src/arrow/flight/flight_sql/client.h +++ b/cpp/src/arrow/flight/flight_sql/client.h @@ -17,15 +17,13 @@ #pragma once -#include #include -#include #include #include #include -#include -namespace pb = arrow::flight::protocol; +#include +#include namespace arrow { namespace flight { @@ -159,13 +157,6 @@ class ARROW_EXPORT FlightSqlClient { arrow::Result> GetSqlInfo(const FlightCallOptions& options, const std::vector& sql_info); - /// \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. @@ -178,21 +169,29 @@ class ARROW_EXPORT FlightSqlClient { class PreparedStatement { std::shared_ptr client_; FlightCallOptions options_; - pb::sql::ActionCreatePreparedStatementResult prepared_statement_result_; + 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] query The query that will be executed. - PreparedStatement( - std::shared_ptr client_, const std::string& query, - pb::sql::ActionCreatePreparedStatementResult& prepared_statement_result, - FlightCallOptions options); + /// \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(std::shared_ptr 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. @@ -205,11 +204,11 @@ class ARROW_EXPORT FlightSqlClient { /// \brief Retrieve the parameter schema from the query. /// \return The parameter schema from the query. - arrow::Result> GetParameterSchema(); + std::shared_ptr parameter_schema() const; /// \brief Retrieve the ResultSet schema from the query. /// \return The ResultSet schema from the query. - arrow::Result> GetResultSetSchema(); + 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. diff --git a/cpp/src/arrow/flight/flight_sql/client_internal.h b/cpp/src/arrow/flight/flight_sql/client_internal.h index 12ac4cd6e1d..23eba399b9d 100644 --- a/cpp/src/arrow/flight/flight_sql/client_internal.h +++ b/cpp/src/arrow/flight/flight_sql/client_internal.h @@ -17,6 +17,8 @@ #include +#include + namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index 2e9886b291f..13851aae461 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -396,7 +396,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client.Prepare(call_options, query)); - ASSERT_OK_AND_ASSIGN(auto parameter_schema, prepared_statement->GetParameterSchema()); + auto parameter_schema = prepared_statement->parameter_schema(); arrow::Int64Builder int_builder; ASSERT_OK(int_builder.Append(1)); @@ -459,10 +459,9 @@ TEST(TestFlightSqlClient, TestGetSqlInfo) { auto client_mock = std::make_shared(); FlightSqlClient sql_client(client_mock); - 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}; + 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); diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc index 43697ae6f44..20b04a050ed 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc @@ -15,8 +15,12 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight_sql/example/sqlite_server.h" - +#include +#include +#include +#include +#include +#include #include #include @@ -25,12 +29,6 @@ #include #include -#include "arrow/api.h" -#include "arrow/flight/flight_sql/example/sqlite_statement.h" -#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" -#include "arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h" -#include "arrow/flight/flight_sql/server.h" - namespace arrow { namespace flight { namespace sql { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h index 8fdcec87d8d..d7c86a4682e 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h @@ -17,16 +17,17 @@ #pragma once +#include +#include +#include +#include #include #include #include #include - -#include "arrow/api.h" -#include "arrow/flight/flight_sql/example/sqlite_statement.h" -#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" -#include "arrow/flight/flight_sql/server.h" +#include +#include namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc index 723f4323efc..76ef4e4ce2a 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc @@ -15,15 +15,12 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight_sql/example/sqlite_statement.h" - +#include +#include #include #include -#include "arrow/api.h" -#include "arrow/flight/flight_sql/example/sqlite_server.h" - namespace arrow { namespace flight { namespace sql { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h index ee9191e42b3..d0285dbd96e 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h @@ -17,9 +17,11 @@ #pragma once +#include #include -#include "arrow/api.h" +#include +#include namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc index 27c3d01816f..f510b319048 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc @@ -15,13 +15,11 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" - +#include +#include +#include #include -#include "arrow/api.h" -#include "arrow/flight/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); \ diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h index 2ae8db6e8e6..336592807dd 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h @@ -17,10 +17,11 @@ #pragma once +#include +#include #include -#include "arrow/api.h" -#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cpp b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cpp index d78684836e4..229057aa230 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cpp +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h" +#include #include #include @@ -24,8 +24,8 @@ #include -#include "arrow/flight/flight_sql/example/sqlite_server.h" -#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include +#include namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h index 0fe7cef7efb..e8f157e287a 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h @@ -17,14 +17,13 @@ #pragma once +#include +#include #include #include -#include -#include - -#include "arrow/flight/flight_sql/example/sqlite_statement.h" -#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" +#include +#include namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index e0bbaeafdfd..90834df1f3c 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -18,30 +18,16 @@ // Interfaces to use for defining Flight RPC servers. API should be considered // experimental for now -#include "arrow/flight/flight_sql/server.h" - +#include +#include #include +#include #include #include #include #include -#include "arrow/api.h" -#include "arrow/buffer.h" - -std::string arrow::flight::sql::FlightSqlServerBase::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); - - const std::string& ticket_string = ticket.SerializeAsString(); - return ticket_string; -} - namespace arrow { namespace flight { namespace sql { @@ -248,6 +234,18 @@ ParseActionClosePreparedStatementRequest(const google::protobuf::Any& any) { return result; } +std::string FlightSqlServerBase::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); + + const std::string& ticket_string = ticket.SerializeAsString(); + return ticket_string; +} + Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, std::unique_ptr* info) { @@ -354,7 +352,9 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, const FlightDescriptor& request = reader->descriptor(); google::protobuf::Any any; - any.ParseFromArray(request.cmd.data(), static_cast(request.cmd.size())); + 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, @@ -395,18 +395,20 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, Status FlightSqlServerBase::ListActions(const ServerCallContext& context, std::vector* actions) { - *actions = {FlightSqlServerBase::FLIGHT_SQL_CREATE_PREPARED_STATEMENT, - FlightSqlServerBase::FLIGHT_SQL_CLOSE_PREPARED_STATEMENT}; + *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::FLIGHT_SQL_CREATE_PREPARED_STATEMENT.type) { + if (action.type == FlightSqlServerBase::kCreatePreparedStatementActionType.type) { google::protobuf::Any any_command; - any_command.ParseFromArray(action.body->data(), - static_cast(action.body->size())); + 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)); @@ -432,10 +434,11 @@ Status FlightSqlServerBase::DoAction(const ServerCallContext& context, *result_stream = std::unique_ptr(new SimpleResultStream({Result{buf}})); return Status::OK(); - } else if (action.type == - FlightSqlServerBase::FLIGHT_SQL_CLOSE_PREPARED_STATEMENT.type) { + } else if (action.type == FlightSqlServerBase::kClosePreparedStatementActionType.type) { google::protobuf::Any any; - any.ParseFromArray(action.body->data(), static_cast(action.body->size())); + 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)); diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index 79bc293a7e0..c13ec5f1256 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -20,45 +20,50 @@ #pragma once +#include +#include +#include #include -#include "arrow/api.h" -#include "arrow/flight/flight_sql/example/sqlite_statement.h" -#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" -#include "arrow/flight/flight_sql/server.h" +#include +#include namespace arrow { namespace flight { namespace sql { -using StatementQuery = struct StatementQuery { std::string query; }; +struct StatementQuery { + std::string query; +}; -using StatementUpdate = struct StatementUpdate { std::string query; }; +struct StatementUpdate { + std::string query; +}; -using StatementQueryTicket = struct StatementQueryTicket { +struct StatementQueryTicket { std::string statement_handle; }; -using PreparedStatementQuery = struct PreparedStatementQuery { +struct PreparedStatementQuery { std::string prepared_statement_handle; }; -using PreparedStatementUpdate = struct PreparedStatementUpdate { +struct PreparedStatementUpdate { std::string prepared_statement_handle; }; -using GetSqlInfo = struct GetSqlInfo { +struct GetSqlInfo { // TODO: To be implemented. }; -using GetSchemas = struct GetSchemas { +struct GetSchemas { bool has_catalog; std::string catalog; bool has_schema_filter_pattern; std::string schema_filter_pattern; }; -using GetTables = struct GetTables { +struct GetTables { bool has_catalog; std::string catalog; bool has_schema_filter_pattern; @@ -69,7 +74,7 @@ using GetTables = struct GetTables { bool include_schema; }; -using GetPrimaryKeys = struct GetPrimaryKeys { +struct GetPrimaryKeys { bool has_catalog; std::string catalog; bool has_schema; @@ -77,7 +82,7 @@ using GetPrimaryKeys = struct GetPrimaryKeys { std::string table; }; -using GetExportedKeys = struct GetExportedKeys { +struct GetExportedKeys { bool has_catalog; std::string catalog; bool has_schema; @@ -85,7 +90,7 @@ using GetExportedKeys = struct GetExportedKeys { std::string table; }; -using GetImportedKeys = struct GetImportedKeys { +struct GetImportedKeys { bool has_catalog; std::string catalog; bool has_schema; @@ -93,7 +98,7 @@ using GetImportedKeys = struct GetImportedKeys { std::string table; }; -using GetCrossReference = struct GetCrossReference { +struct GetCrossReference { bool has_pk_catalog; std::string pk_catalog; bool has_pk_schema; @@ -106,21 +111,21 @@ using GetCrossReference = struct GetCrossReference { std::string fk_table; }; -using ActionCreatePreparedStatementRequest = struct ActionCreatePreparedStatementRequest { +struct ActionCreatePreparedStatementRequest { std::string query; }; -using ActionClosePreparedStatementRequest = struct ActionClosePreparedStatementRequest { +struct ActionClosePreparedStatementRequest { std::string prepared_statement_handle; }; -using ActionCreatePreparedStatementResult = struct ActionCreatePreparedStatementResult { +struct ActionCreatePreparedStatementResult { std::shared_ptr dataset_schema; std::shared_ptr parameter_schema; std::string prepared_statement_handle; }; -class FlightSqlServerBase : public FlightServerBase { +class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { public: Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, std::unique_ptr* info) override; @@ -132,12 +137,12 @@ class FlightSqlServerBase : public FlightServerBase { std::unique_ptr reader, std::unique_ptr writer) override; - const ActionType FLIGHT_SQL_CREATE_PREPARED_STATEMENT = + const ActionType kCreatePreparedStatementActionType = ActionType{"CreatePreparedStatement", "Creates a reusable prepared statement resource on the server.\n" "Request Message: ActionCreatePreparedStatementRequest\n" "Response Message: ActionCreatePreparedStatementResult"}; - const ActionType FLIGHT_SQL_CLOSE_PREPARED_STATEMENT = + const ActionType kClosePreparedStatementActionType = ActionType{"ClosePreparedStatement", "Closes a reusable prepared statement resource on the server.\n" "Request Message: ActionClosePreparedStatementRequest\n" diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index f466b320584..df1c0d971ee 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -17,8 +17,8 @@ #include #include +#include #include -#include #include #include #include @@ -348,7 +348,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) auto prepared_statement, sql_client->Prepare({}, "SELECT * FROM intTable WHERE keyName LIKE ?")); - ASSERT_OK_AND_ASSIGN(auto parameter_schema, prepared_statement->GetParameterSchema()); + auto parameter_schema = prepared_statement->parameter_schema(); const std::shared_ptr& expected_parameter_schema = arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); @@ -428,7 +428,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding sql_client->Prepare( {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', ?)")); - ASSERT_OK_AND_ASSIGN(auto parameter_schema, prepared_statement->GetParameterSchema()); + auto parameter_schema = prepared_statement->parameter_schema(); const std::shared_ptr& expected_parameter_schema = arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); diff --git a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc index c8e3de9fae5..fb2a8890bfb 100644 --- a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc @@ -114,10 +114,10 @@ Status RunMain() { call_options.headers.push_back(bearer_result.ValueOrDie()); } - FlightSqlClient sqlClient(std::move(client)); + FlightSqlClient sql_client(std::move(client)); if (FLAGS_command == "ExecuteUpdate") { - ARROW_ASSIGN_OR_RAISE(auto rows, sqlClient.ExecuteUpdate(call_options, FLAGS_query)); + ARROW_ASSIGN_OR_RAISE(auto rows, sql_client.ExecuteUpdate(call_options, FLAGS_query)); std::cout << "Result: " << rows << std::endl; @@ -127,19 +127,17 @@ Status RunMain() { std::unique_ptr info; if (FLAGS_command == "Execute") { - ARROW_ASSIGN_OR_RAISE(info, sqlClient.Execute(call_options, FLAGS_query)); + ARROW_ASSIGN_OR_RAISE(info, sql_client.Execute(call_options, FLAGS_query)); } else if (FLAGS_command == "GetCatalogs") { - ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetCatalogs(call_options)); + ARROW_ASSIGN_OR_RAISE(info, sql_client.GetCatalogs(call_options)); } else if (FLAGS_command == "PreparedStatementExecute") { ARROW_ASSIGN_OR_RAISE(auto prepared_statement, - sqlClient.Prepare(call_options, FLAGS_query)); + 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, sqlClient.Prepare({}, FLAGS_query)); - ARROW_ASSIGN_OR_RAISE(auto parameter_schema, - prepared_statement->GetParameterSchema()); - ARROW_ASSIGN_OR_RAISE(auto result_set_schema, - prepared_statement->GetResultSetSchema()); + 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; @@ -153,30 +151,30 @@ Status RunMain() { ARROW_ASSIGN_OR_RAISE(info, prepared_statement->Execute()); } else if (FLAGS_command == "GetSchemas") { ARROW_ASSIGN_OR_RAISE( - info, sqlClient.GetSchemas(call_options, &FLAGS_catalog, &FLAGS_schema)); + info, sql_client.GetSchemas(call_options, &FLAGS_catalog, &FLAGS_schema)); } else if (FLAGS_command == "GetTableTypes") { - ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetTableTypes(call_options)); + ARROW_ASSIGN_OR_RAISE(info, sql_client.GetTableTypes(call_options)); } else if (FLAGS_command == "GetTables") { std::vector table_types = {}; bool include_schema = false; - ARROW_ASSIGN_OR_RAISE(info, - sqlClient.GetTables(call_options, &FLAGS_catalog, &FLAGS_schema, - &FLAGS_table, include_schema, table_types)); + ARROW_ASSIGN_OR_RAISE( + info, sql_client.GetTables(call_options, &FLAGS_catalog, &FLAGS_schema, + &FLAGS_table, include_schema, table_types)); } else if (FLAGS_command == "GetExportedKeys") { - ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetExportedKeys(call_options, &FLAGS_catalog, - &FLAGS_schema, FLAGS_table)); + ARROW_ASSIGN_OR_RAISE(info, sql_client.GetExportedKeys(call_options, &FLAGS_catalog, + &FLAGS_schema, FLAGS_table)); } else if (FLAGS_command == "GetImportedKeys") { - ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetImportedKeys(call_options, &FLAGS_catalog, - &FLAGS_schema, FLAGS_table)); + ARROW_ASSIGN_OR_RAISE(info, sql_client.GetImportedKeys(call_options, &FLAGS_catalog, + &FLAGS_schema, FLAGS_table)); } else if (FLAGS_command == "GetPrimaryKeys") { - ARROW_ASSIGN_OR_RAISE(info, sqlClient.GetPrimaryKeys(call_options, &FLAGS_catalog, - &FLAGS_schema, FLAGS_table)); + ARROW_ASSIGN_OR_RAISE(info, sql_client.GetPrimaryKeys(call_options, &FLAGS_catalog, + &FLAGS_schema, FLAGS_table)); } if (info != NULLPTR && !boost::istarts_with(FLAGS_command, "PreparedStatementExecute")) { - return PrintResults(sqlClient, call_options, info); + return PrintResults(sql_client, call_options, info); } return Status::OK(); diff --git a/cpp/src/arrow/flight/flight_sql/test_server_cli.cc b/cpp/src/arrow/flight/flight_sql/test_server_cli.cc index 0e0d63a272d..4ace0877765 100644 --- a/cpp/src/arrow/flight/flight_sql/test_server_cli.cc +++ b/cpp/src/arrow/flight/flight_sql/test_server_cli.cc @@ -15,20 +15,19 @@ // specific language governing permissions and limitations // under the License. +#include +#include +#include +#include +#include +#include +#include #include #include #include #include -#include "arrow/flight/flight_sql/example/sqlite_server.h" -#include "arrow/flight/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"); std::unique_ptr g_server; From c079a477bb849594ff1e899dab4983519bed8d90 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 2 Nov 2021 17:30:40 -0300 Subject: [PATCH 173/237] [C++] Use util::optional on Flight SQL structs (#191) * Use optionals on Flight SQL command structs * Use delete instead of free() on server_test.cc * Un-nest PreparedStatement class from FlightSqlClient * Make PreparedStatement::IsClosed const --- cpp/src/arrow/flight/flight_sql/client.cc | 30 ++--- cpp/src/arrow/flight/flight_sql/client.h | 119 +++++++++--------- .../arrow/flight/flight_sql/client_test.cc | 4 +- .../flight_sql/example/sqlite_server.cc | 54 ++++---- cpp/src/arrow/flight/flight_sql/server.cc | 49 +++----- cpp/src/arrow/flight/flight_sql/server.h | 46 +++---- .../arrow/flight/flight_sql/server_test.cc | 4 +- 7 files changed, 141 insertions(+), 165 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/client.cc b/cpp/src/arrow/flight/flight_sql/client.cc index 8e0fc34f55d..dfc77e1698d 100644 --- a/cpp/src/arrow/flight/flight_sql/client.cc +++ b/cpp/src/arrow/flight/flight_sql/client.cc @@ -39,10 +39,11 @@ FlightSqlClient::FlightSqlClient(std::shared_ptr cli FlightSqlClient::FlightSqlClient(std::unique_ptr client) : impl_(internal::FlightClientImpl_Create(std::move(client))) {} -FlightSqlClient::PreparedStatement::PreparedStatement( - std::shared_ptr client, std::string handle, - std::shared_ptr dataset_schema, std::shared_ptr parameter_schema, - FlightCallOptions options) +PreparedStatement::PreparedStatement(std::shared_ptr client, + std::string handle, + std::shared_ptr dataset_schema, + std::shared_ptr parameter_schema, + FlightCallOptions options) : client_(std::move(client)), options_(std::move(options)), handle_(std::move(handle)), @@ -52,7 +53,7 @@ FlightSqlClient::PreparedStatement::PreparedStatement( FlightSqlClient::~FlightSqlClient() = default; -FlightSqlClient::PreparedStatement::~PreparedStatement() { +PreparedStatement::~PreparedStatement() { if (IsClosed()) return; const Status status = Close(); @@ -260,8 +261,8 @@ arrow::Result> FlightSqlClient::DoGet( return std::move(stream); } -arrow::Result> -FlightSqlClient::Prepare(const FlightCallOptions& options, const std::string& query) { +arrow::Result> FlightSqlClient::Prepare( + const FlightCallOptions& options, const std::string& query) { google::protobuf::Any command; pb::sql::ActionCreatePreparedStatementRequest request; request.set_query(query); @@ -317,7 +318,7 @@ FlightSqlClient::Prepare(const FlightCallOptions& options, const std::string& qu parameter_schema, options); } -arrow::Result> FlightSqlClient::PreparedStatement::Execute() { +arrow::Result> PreparedStatement::Execute() { if (is_closed_) { return Status::Invalid("Statement already closed."); } @@ -352,7 +353,7 @@ arrow::Result> FlightSqlClient::PreparedStatement::E return std::move(info); } -arrow::Result FlightSqlClient::PreparedStatement::ExecuteUpdate() { +arrow::Result PreparedStatement::ExecuteUpdate() { if (is_closed_) { return Status::Invalid("Statement already closed."); } @@ -389,24 +390,23 @@ arrow::Result FlightSqlClient::PreparedStatement::ExecuteUpdate() { return result.record_count(); } -Status FlightSqlClient::PreparedStatement::SetParameters( - std::shared_ptr parameter_binding) { +Status PreparedStatement::SetParameters(std::shared_ptr parameter_binding) { parameter_binding_ = std::move(parameter_binding); return Status::OK(); } -bool FlightSqlClient::PreparedStatement::IsClosed() { return is_closed_; } +bool PreparedStatement::IsClosed() const { return is_closed_; } -std::shared_ptr FlightSqlClient::PreparedStatement::dataset_schema() const { +std::shared_ptr PreparedStatement::dataset_schema() const { return dataset_schema_; } -std::shared_ptr FlightSqlClient::PreparedStatement::parameter_schema() const { +std::shared_ptr PreparedStatement::parameter_schema() const { return parameter_schema_; } -Status FlightSqlClient::PreparedStatement::Close() { +Status PreparedStatement::Close() { if (is_closed_) { return Status::Invalid("Statement already closed."); } diff --git a/cpp/src/arrow/flight/flight_sql/client.h b/cpp/src/arrow/flight/flight_sql/client.h index 218e949ed89..20c8dbd30e6 100644 --- a/cpp/src/arrow/flight/flight_sql/client.h +++ b/cpp/src/arrow/flight/flight_sql/client.h @@ -33,14 +33,14 @@ namespace internal { class FlightClientImpl; } +class PreparedStatement; + /// \brief Flight client with Flight SQL semantics. class ARROW_EXPORT FlightSqlClient { private: std::shared_ptr impl_; public: - class PreparedStatement; - explicit FlightSqlClient(std::shared_ptr client); ~FlightSqlClient(); @@ -165,65 +165,64 @@ class ARROW_EXPORT FlightSqlClient { const FlightCallOptions& options, const std::string& query); explicit FlightSqlClient(std::unique_ptr client); +}; - class PreparedStatement { - std::shared_ptr 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(std::shared_ptr 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 Closes the prepared statement. - /// \param[in] options RPC-layer hints for this call. - /// \return Status. - Status Close(); - - /// \brief Checks if the prepared statement is closed. - /// \return The state of the prepared statement. - bool IsClosed(); - }; +class PreparedStatement { + std::shared_ptr 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(std::shared_ptr 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 Closes the prepared statement. + /// \param[in] options RPC-layer hints for this call. + /// \return Status. + Status Close(); + + /// \brief Checks if the prepared statement is closed. + /// \return The state of the prepared statement. + bool IsClosed() const; }; } // namespace sql diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index 13851aae461..b7ead31c2a0 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -537,7 +537,7 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateNoParameterBinding) { AssertTestPreparedStatementExecuteUpdateOk( - [](const std::shared_ptr& prepared_statement, + [](const std::shared_ptr& prepared_statement, internal::FlightClientImpl& client_mock, const std::shared_ptr* schema, const int64_t& row_count) {}, NULLPTR); @@ -547,7 +547,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding const auto schema = arrow::schema( {arrow::field("field0", arrow::utf8()), arrow::field("field1", arrow::uint8())}); AssertTestPreparedStatementExecuteUpdateOk( - [](const std::shared_ptr& prepared_statement, + [](const std::shared_ptr& prepared_statement, internal::FlightClientImpl& client_mock, const std::shared_ptr* schema, const int64_t& row_count) { std::shared_ptr string_array; diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc index 20b04a050ed..a0c0b535c4e 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc @@ -61,16 +61,18 @@ std::string PrepareQueryForGetTables(const GetTables& command) { 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.has_catalog) { - table_query << " and catalog_name='" << command.catalog << "'"; + if (command.catalog.has_value()) { + table_query << " and catalog_name='" << command.catalog.value() << "'"; } - if (command.has_schema_filter_pattern) { - table_query << " and schema_name LIKE '" << command.schema_filter_pattern << "'"; + if (command.schema_filter_pattern.has_value()) { + table_query << " and schema_name LIKE '" << command.schema_filter_pattern.value() + << "'"; } - if (command.has_table_name_filter_pattern) { - table_query << " and table_name LIKE '" << command.table_name_filter_pattern << "'"; + 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()) { @@ -535,12 +537,12 @@ Status SQLiteFlightSqlServer::DoGetPrimaryKeys( "table_name, type as table_type\n" "FROM sqlite_master) where 1=1 and pk != 0"; - if (command.has_catalog) { - table_query << " and catalog_name LIKE '" << command.catalog << "'"; + if (command.catalog.has_value()) { + table_query << " and catalog_name LIKE '" << command.catalog.value() << "'"; } - if (command.has_schema) { - table_query << " and schema_name LIKE '" << command.schema << "'"; + if (command.schema.has_value()) { + table_query << " and schema_name LIKE '" << command.schema.value() << "'"; } table_query << " and table_name LIKE '" << command.table << "'"; @@ -592,11 +594,11 @@ Status SQLiteFlightSqlServer::DoGetImportedKeys( const GetImportedKeys& command, const ServerCallContext& context, std::unique_ptr* result) { std::string filter = "fk_table_name = '" + command.table + "'"; - if (command.has_catalog) { - filter += " AND fk_catalog_name = '" + command.catalog + "'"; + if (command.catalog.has_value()) { + filter += " AND fk_catalog_name = '" + command.catalog.value() + "'"; } - if (command.has_schema) { - filter += " AND fk_schema_name = '" + command.schema + "'"; + if (command.schema.has_value()) { + filter += " AND fk_schema_name = '" + command.schema.value() + "'"; } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); @@ -613,11 +615,11 @@ Status SQLiteFlightSqlServer::DoGetExportedKeys( const GetExportedKeys& command, const ServerCallContext& context, std::unique_ptr* result) { std::string filter = "pk_table_name = '" + command.table + "'"; - if (command.has_catalog) { - filter += " AND pk_catalog_name = '" + command.catalog + "'"; + if (command.catalog.has_value()) { + filter += " AND pk_catalog_name = '" + command.catalog.value() + "'"; } - if (command.has_schema) { - filter += " AND pk_schema_name = '" + command.schema + "'"; + if (command.schema.has_value()) { + filter += " AND pk_schema_name = '" + command.schema.value() + "'"; } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); @@ -634,19 +636,19 @@ Status SQLiteFlightSqlServer::DoGetCrossReference( const GetCrossReference& command, const ServerCallContext& context, std::unique_ptr* result) { std::string filter = "pk_table_name = '" + command.pk_table + "'"; - if (command.has_pk_catalog) { - filter += " AND pk_catalog_name = '" + command.pk_catalog + "'"; + if (command.pk_catalog.has_value()) { + filter += " AND pk_catalog_name = '" + command.pk_catalog.value() + "'"; } - if (command.has_pk_schema) { - filter += " AND pk_schema_name = '" + command.pk_schema + "'"; + if (command.pk_schema.has_value()) { + filter += " AND pk_schema_name = '" + command.pk_schema.value() + "'"; } filter += " AND fk_table_name = '" + command.fk_table + "'"; - if (command.has_fk_catalog) { - filter += " AND fk_catalog_name = '" + command.fk_catalog + "'"; + if (command.fk_catalog.has_value()) { + filter += " AND fk_catalog_name = '" + command.fk_catalog.value() + "'"; } - if (command.has_fk_schema) { - filter += " AND fk_schema_name = '" + command.fk_schema + "'"; + if (command.fk_schema.has_value()) { + filter += " AND fk_schema_name = '" + command.fk_schema.value() + "'"; } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index 90834df1f3c..347b0e3bec6 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -28,6 +28,9 @@ #include #include +#define PROPERTY_TO_OPTIONAL(COMMAND, PROPERTY) \ + COMMAND.has_##PROPERTY() ? util::make_optional(COMMAND.PROPERTY()) : util::nullopt; + namespace arrow { namespace flight { namespace sql { @@ -42,15 +45,11 @@ arrow::Result ParseCommandGetCrossReference( } GetCrossReference result; - result.has_pk_catalog = command.has_pk_catalog(); - result.pk_catalog = command.pk_catalog(); - result.has_pk_schema = command.has_pk_schema(); - result.pk_schema = command.pk_schema(); + result.pk_catalog = PROPERTY_TO_OPTIONAL(command, pk_catalog); + result.pk_schema = PROPERTY_TO_OPTIONAL(command, pk_schema); result.pk_table = command.pk_table(); - result.has_fk_catalog = command.has_fk_catalog(); - result.fk_catalog = command.fk_catalog(); - result.has_fk_schema = command.has_fk_schema(); - result.fk_schema = command.fk_schema(); + result.fk_catalog = PROPERTY_TO_OPTIONAL(command, fk_catalog); + result.fk_schema = PROPERTY_TO_OPTIONAL(command, fk_schema); result.fk_table = command.fk_table(); return result; } @@ -63,10 +62,8 @@ arrow::Result ParseCommandGetImportedKeys( } GetImportedKeys result; - result.has_catalog = command.has_catalog(); - result.catalog = command.catalog(); - result.has_schema = command.has_schema(); - result.schema = command.schema(); + result.catalog = PROPERTY_TO_OPTIONAL(command, catalog); + result.schema = PROPERTY_TO_OPTIONAL(command, schema); result.table = command.table(); return result; } @@ -79,10 +76,8 @@ arrow::Result ParseCommandGetExportedKeys( } GetExportedKeys result; - result.has_catalog = command.has_catalog(); - result.catalog = command.catalog(); - result.has_schema = command.has_schema(); - result.schema = command.schema(); + result.catalog = PROPERTY_TO_OPTIONAL(command, catalog); + result.schema = PROPERTY_TO_OPTIONAL(command, schema); result.table = command.table(); return result; } @@ -95,10 +90,8 @@ arrow::Result ParseCommandGetPrimaryKeys( } GetPrimaryKeys result; - result.has_catalog = command.has_catalog(); - result.catalog = command.catalog(); - result.has_schema = command.has_schema(); - result.schema = command.schema(); + result.catalog = PROPERTY_TO_OPTIONAL(command, catalog); + result.schema = PROPERTY_TO_OPTIONAL(command, schema); result.table = command.table(); return result; } @@ -120,10 +113,8 @@ arrow::Result ParseCommandGetSchemas(const google::protobuf::Any& an } GetSchemas result; - result.has_catalog = command.has_catalog(); - result.catalog = command.catalog(); - result.has_schema_filter_pattern = command.has_schema_filter_pattern(); - result.schema_filter_pattern = command.schema_filter_pattern(); + result.catalog = PROPERTY_TO_OPTIONAL(command, catalog); + result.schema_filter_pattern = PROPERTY_TO_OPTIONAL(command, schema_filter_pattern); return result; } @@ -163,12 +154,10 @@ arrow::Result ParseCommandGetTables(const google::protobuf::Any& any) table_types.push_back(item); } GetTables result; - result.has_catalog = command.has_catalog(); - result.catalog = command.catalog(); - result.has_schema_filter_pattern = command.has_schema_filter_pattern(); - result.schema_filter_pattern = command.schema_filter_pattern(); - result.has_table_name_filter_pattern = command.has_table_name_filter_pattern(); - result.table_name_filter_pattern = command.table_name_filter_pattern(); + result.catalog = PROPERTY_TO_OPTIONAL(command, catalog); + result.schema_filter_pattern = PROPERTY_TO_OPTIONAL(command, 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; diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index c13ec5f1256..bd1a6a297d4 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -24,6 +24,7 @@ #include #include #include +#include #include #include @@ -57,57 +58,42 @@ struct GetSqlInfo { }; struct GetSchemas { - bool has_catalog; - std::string catalog; - bool has_schema_filter_pattern; - std::string schema_filter_pattern; + util::optional catalog; + util::optional schema_filter_pattern; }; struct GetTables { - bool has_catalog; - std::string catalog; - bool has_schema_filter_pattern; - std::string schema_filter_pattern; - bool has_table_name_filter_pattern; - std::string table_name_filter_pattern; + util::optional catalog; + util::optional schema_filter_pattern; + util::optional table_name_filter_pattern; std::vector table_types; bool include_schema; }; struct GetPrimaryKeys { - bool has_catalog; - std::string catalog; - bool has_schema; - std::string schema; + util::optional catalog; + util::optional schema; std::string table; }; struct GetExportedKeys { - bool has_catalog; - std::string catalog; - bool has_schema; - std::string schema; + util::optional catalog; + util::optional schema; std::string table; }; struct GetImportedKeys { - bool has_catalog; - std::string catalog; - bool has_schema; - std::string schema; + util::optional catalog; + util::optional schema; std::string table; }; struct GetCrossReference { - bool has_pk_catalog; - std::string pk_catalog; - bool has_pk_schema; - std::string pk_schema; + util::optional pk_catalog; + util::optional pk_schema; std::string pk_table; - bool has_fk_catalog; - std::string fk_catalog; - bool has_fk_schema; - std::string fk_schema; + util::optional fk_catalog; + util::optional fk_schema; std::string fk_table; }; diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index df1c0d971ee..4581f3799c6 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -62,8 +62,8 @@ class TestFlightSqlServer : public ::testing::Environment { void TearDown() override { server->Stop(); - free(server); - free(sql_client); + delete server; + delete sql_client; } }; From f080caa3739a71992bf11ec37c03485eb604f763 Mon Sep 17 00:00:00 2001 From: Jose Almeida <53087160+jcralmeida@users.noreply.github.com> Date: Tue, 2 Nov 2021 17:42:37 -0300 Subject: [PATCH 174/237] [CPP] Fix on sqlite classes from FlightSqlServer (#181) * Change FlightMetadataWriter and FlightMessageReader from unique_ptr to raw ptr * Change reinterpret_cast to dynamic_cast * Add const to sqlite3Stmt getter * Add const to string parameter * Using ARROW_ASSIGN_OR_RAISE to buffer and batch creation * Change from unique_ptr to raw ptr * Change sqlite_tables_schema_batch_reader extension from cpp to cc * Avoid instantiate a string object * Make ExecuteSql return Status * Change Create methods from sqlStatement to return Result * Fix from rebase * Fix from rebase * Fix server initialization * Fix checkstyle * Add missing ; * Fix docs on methods * add explicit to sqlite_server.h constructor * Fix double free * Add comment of ownership to SQLiteFlightSqlServer constructor * Fixed possible close with null_ptr on sqlite classes * Fix style issues * Use static_cast instead of dynamic_cast * Fix other review comments * Use 'static_cast' when casting scalars on sqlite_server.cc * Fix comments pointed on review Co-authored-by: Rafael Telles --- .../arrow/flight/flight_sql/CMakeLists.txt | 2 +- .../flight_sql/example/sqlite_server.cc | 125 ++++++++++-------- .../flight/flight_sql/example/sqlite_server.h | 20 +-- .../flight_sql/example/sqlite_statement.cc | 23 ++-- .../flight_sql/example/sqlite_statement.h | 10 +- .../example/sqlite_statement_batch_reader.cc | 72 +++++----- .../example/sqlite_statement_batch_reader.h | 16 +-- ...p => sqlite_tables_schema_batch_reader.cc} | 14 +- .../sqlite_tables_schema_batch_reader.h | 9 +- cpp/src/arrow/flight/flight_sql/server.cc | 12 +- cpp/src/arrow/flight/flight_sql/server.h | 10 +- .../flight/flight_sql/test_server_cli.cc | 32 +++-- 12 files changed, 193 insertions(+), 152 deletions(-) rename cpp/src/arrow/flight/flight_sql/example/{sqlite_tables_schema_batch_reader.cpp => sqlite_tables_schema_batch_reader.cc} (89%) diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index d9662817255..247caafe760 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -104,7 +104,7 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) example/sqlite_statement.cc example/sqlite_statement_batch_reader.cc example/sqlite_server.cc - example/sqlite_tables_schema_batch_reader.cpp + example/sqlite_tables_schema_batch_reader.cc "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc") target_link_libraries(flight_sql_test_server PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES} diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc index a0c0b535c4e..86f727762dd 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc @@ -92,8 +92,7 @@ std::string PrepareQueryForGetTables(const GetTables& command) { return table_query.str(); } -Status SetParametersOnSQLiteStatement(sqlite3_stmt* stmt, - std::unique_ptr& reader) { +Status SetParametersOnSQLiteStatement(sqlite3_stmt* stmt, FlightMessageReader* reader) { FlightStreamChunk chunk; while (true) { RETURN_NOT_OK(reader->Next(&chunk)); @@ -108,31 +107,27 @@ Status SetParametersOnSQLiteStatement(sqlite3_stmt* stmt, const std::shared_ptr& column = record_batch->column(c); ARROW_ASSIGN_OR_RAISE(std::shared_ptr scalar, column->GetScalar(i)); - auto& holder = reinterpret_cast(*scalar).value; + auto& holder = static_cast(*scalar).value; switch (holder->type->id()) { case Type::INT64: { - int64_t value = reinterpret_cast(*holder).value; + int64_t value = static_cast(*holder).value; sqlite3_bind_int64(stmt, c + 1, value); break; } case Type::FLOAT: { - double value = reinterpret_cast(*holder).value; + double value = static_cast(*holder).value; sqlite3_bind_double(stmt, c + 1, value); break; } case Type::STRING: { - std::shared_ptr buffer = - reinterpret_cast(*holder).value; - const std::string string = buffer->ToString(); - const char* value = string.c_str(); - sqlite3_bind_text(stmt, c + 1, value, static_cast(strlen(value)), - SQLITE_TRANSIENT); + 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 = - reinterpret_cast(*holder).value; + std::shared_ptr buffer = static_cast(*holder).value; sqlite3_bind_blob(stmt, c + 1, buffer->data(), static_cast(buffer->size()), SQLITE_TRANSIENT); break; @@ -148,53 +143,75 @@ Status SetParametersOnSQLiteStatement(sqlite3_stmt* stmt, return Status::OK(); } -SQLiteFlightSqlServer::SQLiteFlightSqlServer() { - db_ = NULLPTR; - if (sqlite3_open(":memory:", &db_)) { - sqlite3_close(db_); - throw std::runtime_error(std::string("Can't open database: ") + sqlite3_errmsg(db_)); +SQLiteFlightSqlServer::SQLiteFlightSqlServer(sqlite3* db) : db_(db), uuid_generator_({}) { + assert(db_); +} + +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::RError(err_msg); } - ExecuteSql(R"( -CREATE TABLE foreignTable ( - id INTEGER PRIMARY KEY AUTOINCREMENT, - foreignName varchar(100), - value int); + std::shared_ptr result(new SQLiteFlightSqlServer(db)); + + ARROW_UNUSED(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)); + 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 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); + )")); + + return result; } SQLiteFlightSqlServer::~SQLiteFlightSqlServer() { sqlite3_close(db_); } -void SQLiteFlightSqlServer::ExecuteSql(const std::string& sql) { - char* zErrMsg = NULLPTR; - int rc = sqlite3_exec(db_, sql.c_str(), NULLPTR, NULLPTR, &zErrMsg); +Status SQLiteFlightSqlServer::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) { - fprintf(stderr, "SQL error: %s\n", zErrMsg); - sqlite3_free(zErrMsg); + std::string error_msg; + if (err_msg != nullptr) { + error_msg = err_msg; + } + sqlite3_free(err_msg); + return Status::RError(error_msg); } + return Status::OK(); } Status DoGetSQLiteQuery(sqlite3* db, const std::string& query, const std::shared_ptr& schema, std::unique_ptr* result) { std::shared_ptr statement; - ARROW_RETURN_NOT_OK(SqliteStatement::Create(db, query, &statement)); + + ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db, query)); std::shared_ptr reader; - ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create(statement, schema, &reader)); + ARROW_ASSIGN_OR_RAISE(reader, SqliteStatementBatchReader::Create(statement, schema)); *result = std::unique_ptr(new RecordBatchStream(reader)); @@ -220,7 +237,7 @@ Status SQLiteFlightSqlServer::GetFlightInfoStatement(const StatementQuery& comma const std::string& query = command.query; std::shared_ptr statement; - ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, query, &statement)); + ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, query)); std::shared_ptr schema; ARROW_RETURN_NOT_OK(statement->GetSchema(&schema)); @@ -241,10 +258,10 @@ Status SQLiteFlightSqlServer::DoGetStatement(const StatementQueryTicket& command const std::string& sql = command.statement_handle; std::shared_ptr statement; - ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, sql, &statement)); + ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, sql)); std::shared_ptr reader; - ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create(statement, &reader)); + ARROW_ASSIGN_OR_RAISE(reader, SqliteStatementBatchReader::Create(statement)); *result = std::unique_ptr(new RecordBatchStream(reader)); @@ -325,11 +342,11 @@ Status SQLiteFlightSqlServer::DoGetTables(const GetTables& command, std::string query = PrepareQueryForGetTables(command); std::shared_ptr statement; - ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, query, &statement)); + ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, query)); std::shared_ptr reader; - ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create( - statement, SqlSchema::GetTablesSchema(), &reader)); + ARROW_ASSIGN_OR_RAISE(reader, SqliteStatementBatchReader::Create( + statement, SqlSchema::GetTablesSchema())); if (command.include_schema) { std::shared_ptr table_schema_reader = @@ -349,7 +366,7 @@ arrow::Result SQLiteFlightSqlServer::DoPutCommandStatementUpdate( const std::string& sql = command.query; std::shared_ptr statement; - ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, sql, &statement)); + ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, sql)); int64_t record_count; ARROW_RETURN_NOT_OK(statement->ExecuteUpdate(&record_count)); @@ -362,8 +379,7 @@ SQLiteFlightSqlServer::CreatePreparedStatement( const ActionCreatePreparedStatementRequest& request, const ServerCallContext& context) { std::shared_ptr statement; - ARROW_RETURN_NOT_OK(SqliteStatement::Create(db_, request.query, &statement)); - + ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, request.query)); boost::uuids::uuid uuid = uuid_generator_(); prepared_statements_[uuid] = statement; @@ -452,7 +468,7 @@ Status SQLiteFlightSqlServer::DoGetPreparedStatement( std::shared_ptr statement = search->second; std::shared_ptr reader; - ARROW_RETURN_NOT_OK(SqliteStatementBatchReader::Create(statement, &reader)); + ARROW_ASSIGN_OR_RAISE(reader, SqliteStatementBatchReader::Create(statement)); *result = std::unique_ptr(new RecordBatchStream(reader)); @@ -475,8 +491,7 @@ Status SQLiteFlightSqlServer::GetStatementByHandle( Status SQLiteFlightSqlServer::DoPutPreparedStatementQuery( const PreparedStatementQuery& command, const ServerCallContext& context, - std::unique_ptr& reader, - std::unique_ptr& writer) { + FlightMessageReader* reader, FlightMetadataWriter* writer) { const std::string& prepared_statement_handle = command.prepared_statement_handle; std::shared_ptr statement; ARROW_RETURN_NOT_OK(GetStatementByHandle(prepared_statement_handle, &statement)); @@ -489,7 +504,7 @@ Status SQLiteFlightSqlServer::DoPutPreparedStatementQuery( arrow::Result SQLiteFlightSqlServer::DoPutPreparedStatementUpdate( const PreparedStatementUpdate& command, const ServerCallContext& context, - std::unique_ptr& reader) { + FlightMessageReader* reader) { const std::string& prepared_statement_handle = command.prepared_statement_handle; std::shared_ptr statement; diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h index d7c86a4682e..99d1132c85d 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h @@ -54,13 +54,13 @@ inline std::shared_ptr GetUnknownColumnDataType() { /// database. class SQLiteFlightSqlServer : public FlightSqlServerBase { public: - SQLiteFlightSqlServer(); - ~SQLiteFlightSqlServer() override; + static arrow::Result> Create(); + /// \brief Auxiliary method used to execute an arbitrary SQL statement on the underlying /// SQLite database. - void ExecuteSql(const std::string& sql); + Status ExecuteSql(const std::string& sql); Status GetFlightInfoStatement(const StatementQuery& command, const ServerCallContext& context, @@ -96,13 +96,13 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { Status DoGetPreparedStatement(const PreparedStatementQuery& command, const ServerCallContext& context, std::unique_ptr* result) override; - Status DoPutPreparedStatementQuery( - const PreparedStatementQuery& command, const ServerCallContext& context, - std::unique_ptr& reader, - std::unique_ptr& writer) override; + Status DoPutPreparedStatementQuery(const PreparedStatementQuery& command, + const ServerCallContext& context, + FlightMessageReader* reader, + FlightMetadataWriter* writer) override; arrow::Result DoPutPreparedStatementUpdate( const PreparedStatementUpdate& command, const ServerCallContext& context, - std::unique_ptr& reader) override; + FlightMessageReader* reader) override; Status GetFlightInfoTables(const GetTables& command, const ServerCallContext& context, const FlightDescriptor& descriptor, @@ -150,6 +150,10 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { boost::uuids::random_generator uuid_generator_; std::map> prepared_statements_; + /// SQLiteFlightSqlServer + /// \param db The db parameter from SQLite. The Server it is taking the ownership. + explicit SQLiteFlightSqlServer(sqlite3* db); + Status GetStatementByHandle(const std::string& prepared_statement_handle, std::shared_ptr* result); }; diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc index 76ef4e4ce2a..496841f8d5c 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc @@ -42,19 +42,26 @@ std::shared_ptr GetDataTypeFromSqliteType(const int column_type) { } } -Status SqliteStatement::Create(sqlite3* db, const std::string& sql, - std::shared_ptr* result) { - sqlite3_stmt* stmt; +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) { - sqlite3_finalize(stmt); - return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db)); + 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::RError(err_msg); } - result->reset(new SqliteStatement(db, stmt)); - return Status::OK(); + std::shared_ptr result(new SqliteStatement(db, stmt)); + return result; } Status SqliteStatement::GetSchema(std::shared_ptr* schema) const { @@ -114,7 +121,7 @@ Status SqliteStatement::Reset(int* rc) { return Status::OK(); } -sqlite3_stmt* SqliteStatement::GetSqlite3Stmt() { return stmt_; } +sqlite3_stmt* SqliteStatement::GetSqlite3Stmt() const { return stmt_; } Status SqliteStatement::ExecuteUpdate(int64_t* result) { int rc; diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h index d0285dbd96e..39430fb43df 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h @@ -33,10 +33,9 @@ class SqliteStatement { /// \brief Creates a SQLite3 statement. /// \param[in] db SQLite3 database instance. /// \param[in] sql SQL statement. - /// \param[out] result The resulting SqliteStatement. - /// \return Status. - static Status Create(sqlite3* db, const std::string& sql, - std::shared_ptr* result); + /// \return A SqliteStatement object. + static arrow::Result> Create(sqlite3* db, + const std::string& sql); ~SqliteStatement(); @@ -56,7 +55,8 @@ class SqliteStatement { Status Reset(int* rc); /// \brief Returns the underlying sqlite3_stmt. - sqlite3_stmt* GetSqlite3Stmt(); + /// \return A sqlite statement. + sqlite3_stmt* GetSqlite3Stmt() const; /// \brief Executes an UPDATE, INSERT or DELETE statement. /// \param[out] result The number of rows changed by execution. diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc index f510b319048..9b9e193cc33 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc @@ -20,17 +20,18 @@ #include #include -#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((dynamic_cast(builder)).AppendNull()); \ - break; \ - } \ - ARROW_RETURN_NOT_OK( \ - (dynamic_cast(builder)).Append(string, bytes)); \ - break; \ +#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) \ @@ -38,26 +39,29 @@ int bytes = sqlite3_column_bytes(STMT, COLUMN); \ const void* blob = sqlite3_column_blob(STMT, COLUMN); \ if (blob == nullptr) { \ - ARROW_RETURN_NOT_OK((dynamic_cast(builder)).AppendNull()); \ + ARROW_RETURN_NOT_OK( \ + (reinterpret_cast(builder)).AppendNull()); \ break; \ } \ ARROW_RETURN_NOT_OK( \ - (dynamic_cast(builder)).Append((char*)blob, bytes)); \ + (reinterpret_cast(builder)).Append((char*)blob, bytes)); \ break; \ } -#define INT_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ - case TYPE_CLASS##Type::type_id: { \ - sqlite3_int64 value = sqlite3_column_int64(STMT, COLUMN); \ - ARROW_RETURN_NOT_OK((dynamic_cast(builder)).Append(value)); \ - break; \ +#define INT_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ + case TYPE_CLASS##Type::type_id: { \ + 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: { \ - double value = sqlite3_column_double(STMT, COLUMN); \ - ARROW_RETURN_NOT_OK((dynamic_cast(builder)).Append(value)); \ - break; \ +#define FLOAT_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ + case TYPE_CLASS##Type::type_id: { \ + double value = sqlite3_column_double(STMT, COLUMN); \ + ARROW_RETURN_NOT_OK( \ + (reinterpret_cast(builder)).Append(value)); \ + break; \ } #define MAX_BATCH_SIZE 1024 @@ -76,27 +80,27 @@ SqliteStatementBatchReader::SqliteStatementBatchReader( rc_(SQLITE_OK), already_executed_(false) {} -Status SqliteStatementBatchReader::Create( - const std::shared_ptr& statement_, - std::shared_ptr* result) { +Result> SqliteStatementBatchReader::Create( + const std::shared_ptr& statement_) { int rc; ARROW_RETURN_NOT_OK(statement_->Step(&rc)); std::shared_ptr schema; ARROW_RETURN_NOT_OK(statement_->GetSchema(&schema)); - result->reset(new SqliteStatementBatchReader(statement_, schema)); + std::shared_ptr result( + new SqliteStatementBatchReader(statement_, schema)); - return Status::OK(); + return result; } -Status SqliteStatementBatchReader::Create( - const std::shared_ptr& statement_, - const std::shared_ptr& schema, - std::shared_ptr* result) { - result->reset(new SqliteStatementBatchReader(statement_, schema)); +arrow::Result> +SqliteStatementBatchReader::Create(const std::shared_ptr& statement, + const std::shared_ptr& schema) { + std::shared_ptr result( + new SqliteStatementBatchReader(statement, schema)); - return Status::OK(); + return result; } Status SqliteStatementBatchReader::ReadNext(std::shared_ptr* out) { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h index 336592807dd..8bd8c72cb09 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h @@ -32,19 +32,17 @@ class SqliteStatementBatchReader : public RecordBatchReader { public: /// \brief Creates a RecordBatchReader backed by a SQLite statement. /// \param[in] statement SQLite statement to be read. - /// \param[out] result The resulting RecordBatchReader. - /// \return Status. - static Status Create(const std::shared_ptr& statement, - std::shared_ptr* result); + /// \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. - /// \param[out] result The resulting RecordBatchReader. - /// \return Status. - static Status Create(const std::shared_ptr& statement, - const std::shared_ptr& schema, - std::shared_ptr* result); + /// \return A SqliteStatementBatchReader.. + static arrow::Result> Create( + const std::shared_ptr& statement, + const std::shared_ptr& schema); std::shared_ptr schema() const override; diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cpp b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc similarity index 89% rename from cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cpp rename to cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc index 229057aa230..1899b912b89 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cpp +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc @@ -44,8 +44,8 @@ Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* << "JOIN(" << main_query_ << ") order by table_name"; std::shared_ptr schema_statement; - ARROW_RETURN_NOT_OK( - example::SqliteStatement::Create(db_, schema_query.str(), &schema_statement)); + ARROW_ASSIGN_OR_RAISE(schema_statement, + example::SqliteStatement::Create(db_, schema_query.str())) std::shared_ptr first_batch; @@ -83,17 +83,19 @@ Status SqliteTablesWithSchemaBatchReader::ReadNext(std::shared_ptr* } 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(value.ValueOrDie()->data(), value.ValueOrDie()->size())); + schema_builder.Append(schema_buffer->data(), schema_buffer->size())); } std::shared_ptr schema_array; ARROW_RETURN_NOT_OK(schema_builder.Finish(&schema_array)); - auto result = first_batch->AddColumn(4, "table_schema", schema_array); - - ARROW_ASSIGN_OR_RAISE(*batch, result); + ARROW_ASSIGN_OR_RAISE(*batch, first_batch->AddColumn(4, "table_schema", schema_array)); return Status::OK(); } diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h index e8f157e287a..99c7486d0e3 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h @@ -39,11 +39,12 @@ class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { public: /// Constructor for SqliteTablesWithSchemaBatchReader class /// \param reader an shared_ptr from a SqliteStatementBatchReader. - /// \param db_ a pointer to the sqlite3 db. + /// \param 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_(main_query), db_(db) {} + 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; diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index 347b0e3bec6..19e400aa4f1 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -362,12 +362,13 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command, ParseCommandPreparedStatementQuery(any)); - return DoPutPreparedStatementQuery(internal_command, context, reader, writer); + return DoPutPreparedStatementQuery(internal_command, context, 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(internal_command, context, reader)) + ARROW_ASSIGN_OR_RAISE(auto record_count, DoPutPreparedStatementUpdate( + internal_command, context, reader.get())) pb::sql::DoPutUpdateResult result; result.set_record_count(record_count); @@ -589,14 +590,13 @@ Status FlightSqlServerBase::ClosePreparedStatement( Status FlightSqlServerBase::DoPutPreparedStatementQuery( const PreparedStatementQuery& command, const ServerCallContext& context, - std::unique_ptr& reader, - std::unique_ptr& writer) { + FlightMessageReader* reader, FlightMetadataWriter* writer) { return Status::NotImplemented("DoPutPreparedStatementQuery not implemented"); } arrow::Result FlightSqlServerBase::DoPutPreparedStatementUpdate( const PreparedStatementUpdate& command, const ServerCallContext& context, - std::unique_ptr& reader) { + FlightMessageReader* reader) { return Status::NotImplemented("DoPutPreparedStatementUpdate not implemented"); } diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index bd1a6a297d4..2e667311fc4 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -414,10 +414,10 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \param[in] context The call context. /// \param[in] reader A sequence of uploaded record batches. /// \param[in] writer Send metadata back to the client. - virtual Status DoPutPreparedStatementQuery( - const PreparedStatementQuery& command, const ServerCallContext& context, - std::unique_ptr& reader, - std::unique_ptr& writer); + virtual Status DoPutPreparedStatementQuery(const PreparedStatementQuery& command, + const ServerCallContext& context, + FlightMessageReader* reader, + FlightMetadataWriter* writer); /// \brief Executes an update SQL prepared statement. /// \param[in] command The PreparedStatementUpdate object containing the @@ -427,7 +427,7 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \return The changed record count. virtual arrow::Result DoPutPreparedStatementUpdate( const PreparedStatementUpdate& command, const ServerCallContext& context, - std::unique_ptr& reader); + FlightMessageReader* reader); protected: static std::string CreateStatementQueryTicket(const std::string& statement_handle); diff --git a/cpp/src/arrow/flight/flight_sql/test_server_cli.cc b/cpp/src/arrow/flight/flight_sql/test_server_cli.cc index 4ace0877765..220a975a6d8 100644 --- a/cpp/src/arrow/flight/flight_sql/test_server_cli.cc +++ b/cpp/src/arrow/flight/flight_sql/test_server_cli.cc @@ -30,23 +30,33 @@ DEFINE_int32(port, 31337, "Server port to listen on"); -std::unique_ptr g_server; - -int main(int argc, char** argv) { - gflags::SetUsageMessage("Integration testing server for Flight SQL."); - gflags::ParseCommandLineFlags(&argc, &argv, true); - +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); - g_server.reset(new arrow::flight::sql::example::SQLiteFlightSqlServer()); + std::shared_ptr server; + ARROW_ASSIGN_OR_RAISE(server, + arrow::flight::sql::example::SQLiteFlightSqlServer::Create()) - ARROW_CHECK_OK(g_server->Init(options)); + ARROW_CHECK_OK(server->Init(options)); // Exit with a clean error code (0) on SIGTERM - ARROW_CHECK_OK(g_server->SetShutdownOnSignals({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); - std::cout << "Server listening on localhost:" << g_server->port() << std::endl; - ARROW_CHECK_OK(g_server->Serve()); + arrow::Status st = RunMain(); + if (!st.ok()) { + std::cerr << st << std::endl; + return 1; + } return 0; } From 92c546444714402dbf7e25f6643c08951e98c800 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 3 Nov 2021 13:17:52 -0300 Subject: [PATCH 175/237] Remove unused variable on CMakeLists.txt --- cpp/src/arrow/flight/flight_sql/CMakeLists.txt | 4 +--- .../flight_sql/example/sqlite_tables_schema_batch_reader.cc | 6 ++---- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index 247caafe760..fd453ccc5eb 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -55,10 +55,8 @@ add_arrow_lib(arrow_flight_sql ${ARROW_VERSION_SCRIPT_FLAGS} # Defined in cpp/arrow/CMakeLists.txt SHARED_LINK_LIBS arrow_flight_shared - ${ARROW_FLIGHT_SQL_LINK_LIBS} STATIC_LINK_LIBS - arrow_flight_static - ${ARROW_FLIGHT_SQL_LINK_LIBS}) + arrow_flight_static) if(ARROW_TEST_LINKAGE STREQUAL "static") set(ARROW_FLIGHT_SQL_TEST_LINK_LIBS diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc index 1899b912b89..42c28ae1a10 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc @@ -15,8 +15,9 @@ // specific language governing permissions and limitations // under the License. +#include +#include #include - #include #include #include @@ -24,9 +25,6 @@ #include -#include -#include - namespace arrow { namespace flight { namespace sql { From 9716a20c949717eeb3db064df9ee15184d0bea2d Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 3 Nov 2021 16:12:07 -0300 Subject: [PATCH 176/237] Fix issues reported by cppcheck --- cpp/cmake_modules/FindArrowFlightSql.cmake | 2 +- cpp/src/arrow/flight/flight_sql/server.cc | 9 ++++----- cpp/src/arrow/flight/flight_sql/server_test.cc | 1 - 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/cpp/cmake_modules/FindArrowFlightSql.cmake b/cpp/cmake_modules/FindArrowFlightSql.cmake index a40af25c2a4..35141749b1f 100644 --- a/cpp/cmake_modules/FindArrowFlightSql.cmake +++ b/cpp/cmake_modules/FindArrowFlightSql.cmake @@ -51,7 +51,7 @@ if(ARROW_FOUND) arrow_find_package(ARROW_FLIGHT_SQL "${ARROW_HOME}" arrow_flight_sql - arrow/flight/flight-sql/api.h + arrow/flight/flight_sql/api.h ArrowFlightSql arrow-flight-sql) if(NOT ARROW_FLIGHT_SQL_VERSION) diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index 19e400aa4f1..91c09ae878b 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -148,11 +148,10 @@ arrow::Result ParseCommandGetTables(const google::protobuf::Any& any) return Status::Invalid("Unable to unpack CommandGetTables."); } - std::vector table_types; - table_types.reserve(command.table_types_size()); - for (const auto& item : command.table_types()) { - table_types.push_back(item); - } + 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.schema_filter_pattern = PROPERTY_TO_OPTIONAL(command, schema_filter_pattern); diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 4581f3799c6..6da76632d5e 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -505,7 +505,6 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { } TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { - std::vector table_types; ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetPrimaryKeys({}, nullptr, nullptr, "int%")); From 697cde3dab946f727d56d7a0cbdec68ce7b9a301 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 3 Nov 2021 17:54:35 -0300 Subject: [PATCH 177/237] [C++] Ensure client_test.cc does not violate ODR (#192) * Ensure client_test.cc does not violate ODR * Format CMakeLists.txt --- .../arrow/flight/flight_sql/CMakeLists.txt | 10 +++++++-- .../arrow/flight/flight_sql/client_test.cc | 21 ++++++++++++------- 2 files changed, 21 insertions(+), 10 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index fd453ccc5eb..39b3d49047a 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -59,10 +59,15 @@ add_arrow_lib(arrow_flight_sql arrow_flight_static) if(ARROW_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}) 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_TEST_LINK_LIBS arrow_flight_shared arrow_flight_testing_shared + ${ARROW_TEST_LINK_LIBS}) set(ARROW_FLIGHT_SQL_TEST_LINK_LIBS arrow_flight_sql_shared arrow_flight_testing_shared ${ARROW_TEST_LINK_LIBS}) endif() @@ -74,10 +79,11 @@ target_link_libraries(flight_sql_test_app PRIVATE arrow_flight_sql_static add_arrow_test(flight_sql_client_test SOURCES - client_test.cc "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" + client_test.cc + client.cc STATIC_LINK_LIBS - ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} + ${ARROW_FLIGHT_TEST_LINK_LIBS} LABELS "arrow_flight_sql") diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index b7ead31c2a0..ada141913d6 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -54,31 +54,36 @@ class FlightClientImpl { std::unique_ptr* results)); }; -Status FlightClientImpl_GetFlightInfo(FlightClientImpl* client, +Status FlightClientImpl_GetFlightInfo(FlightClientImpl& client, const FlightCallOptions& options, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return client->GetFlightInfo(options, descriptor, info); + return client.GetFlightInfo(options, descriptor, info); } -Status FlightClientImpl_DoPut(FlightClientImpl* client, const FlightCallOptions& options, +Status FlightClientImpl_DoPut(FlightClientImpl& client, const FlightCallOptions& options, const FlightDescriptor& descriptor, const std::shared_ptr& schema, std::unique_ptr* stream, std::unique_ptr* reader) { - return client->DoPut(options, descriptor, schema, stream, reader); + return client.DoPut(options, descriptor, schema, stream, reader); } -Status FlightClientImpl_DoGet(FlightClientImpl* client, const FlightCallOptions& options, +Status FlightClientImpl_DoGet(FlightClientImpl& client, const FlightCallOptions& options, const Ticket& ticket, std::unique_ptr* stream) { - return client->DoGet(options, ticket, stream); + return client.DoGet(options, ticket, stream); } -Status FlightClientImpl_DoAction(FlightClientImpl* client, +Status FlightClientImpl_DoAction(FlightClientImpl& client, const FlightCallOptions& options, const Action& action, std::unique_ptr* results) { - return client->DoAction(options, action, results); + return client.DoAction(options, action, results); +} + +std::shared_ptr FlightClientImpl_Create( + std::unique_ptr client) { + return std::make_shared(); } } // namespace internal From 9ef1cae7645fc4456d55d6bd8e365368b69b8533 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Wed, 27 Oct 2021 11:02:19 -0300 Subject: [PATCH 178/237] Fix some issues on CMake and Server Test --- cpp/src/arrow/flight/flight_sql/CMakeLists.txt | 5 ++--- cpp/src/arrow/flight/flight_sql/server_test.cc | 1 - 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index 39b3d49047a..eeaad62e363 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -74,7 +74,7 @@ endif() add_executable(flight_sql_test_app test_app_cli.cc ${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc) -target_link_libraries(flight_sql_test_app PRIVATE arrow_flight_sql_static +target_link_libraries(flight_sql_test_app PRIVATE ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} ${GFLAGS_LIBRARIES}) add_arrow_test(flight_sql_client_test @@ -100,7 +100,6 @@ add_arrow_test(flight_sql_server_test # Build test server for unit tests if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) find_package(SQLite3) - include_directories(${SQLite3_INCLUDE_DIRS}) add_executable(flight_sql_test_server server.cc @@ -111,7 +110,7 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) example/sqlite_tables_schema_batch_reader.cc "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc") target_link_libraries(flight_sql_test_server - PRIVATE arrow_flight_sql_static ${GFLAGS_LIBRARIES} + PRIVATE ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} ${GFLAGS_LIBRARIES} ${SQLite3_LIBRARIES}) if(ARROW_BUILD_TESTS) diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 6da76632d5e..b1d7660a335 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -44,7 +44,6 @@ class TestFlightSqlServer : public ::testing::Environment { void SetUp() override { server = new TestServer("flight_sql_test_server"); server->Start(); - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); ASSERT_TRUE(server->IsRunning()); std::stringstream ss; From 39f63c24e41a8f9d83793696e38435dc83890e87 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Thu, 28 Oct 2021 14:55:12 -0300 Subject: [PATCH 179/237] Made fixture to client_test.cc to avoid duplication --- .../arrow/flight/flight_sql/client_test.cc | 123 ++++++------------ .../arrow/flight/flight_sql/server_test.cc | 6 + 2 files changed, 48 insertions(+), 81 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index ada141913d6..c61b247d5cd 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -88,6 +88,22 @@ std::shared_ptr FlightClientImpl_Create( } // namespace internal +std::shared_ptr client_mock; +FlightCallOptions call_options; + +class TestFlightSqlClient : public ::testing::Test { +protected: + FlightSqlClient *sql_client; + void SetUp() override { + client_mock = std::make_shared(); + sql_client = new FlightSqlClient(client_mock); + } + + void TearDown() override { + free(sql_client); + } +}; + class FlightMetadataReaderMock : public FlightMetadataReader { public: std::shared_ptr* buffer; @@ -141,11 +157,7 @@ FlightDescriptor getDescriptor(google::protobuf::Message& command) { return FlightDescriptor::Command(string); } -TEST(TestFlightSqlClient, TestGetCatalogs) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - FlightCallOptions call_options; - +TEST_F(TestFlightSqlClient, TestGetCatalogs) { pb::sql::CommandGetCatalogs command; FlightDescriptor descriptor = getDescriptor(command); @@ -154,11 +166,7 @@ TEST(TestFlightSqlClient, TestGetCatalogs) { ASSERT_OK(sql_client.GetCatalogs(call_options)); } -TEST(TestFlightSqlClient, TestGetSchemas) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - FlightCallOptions call_options; - +TEST_F(TestFlightSqlClient, TestGetSchemas) { std::string schema_filter_pattern = "schema_filter_pattern"; std::string catalog = "catalog"; @@ -172,11 +180,7 @@ TEST(TestFlightSqlClient, TestGetSchemas) { ASSERT_OK(sql_client.GetSchemas(call_options, &catalog, &schema_filter_pattern)); } -TEST(TestFlightSqlClient, TestGetTables) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - FlightCallOptions call_options; - +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"; @@ -200,11 +204,7 @@ TEST(TestFlightSqlClient, TestGetTables) { table_types)); } -TEST(TestFlightSqlClient, TestGetTableTypes) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - FlightCallOptions call_options; - +TEST_F(TestFlightSqlClient, TestGetTableTypes) { pb::sql::CommandGetTableTypes command; FlightDescriptor descriptor = getDescriptor(command); @@ -213,11 +213,7 @@ TEST(TestFlightSqlClient, TestGetTableTypes) { ASSERT_OK(sql_client.GetTableTypes(call_options)); } -TEST(TestFlightSqlClient, TestGetExported) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - FlightCallOptions call_options; - +TEST_F(TestFlightSqlClient, TestGetExported) { std::string catalog = "catalog"; std::string schema = "schema"; std::string table = "table"; @@ -233,11 +229,7 @@ TEST(TestFlightSqlClient, TestGetExported) { ASSERT_OK(sql_client.GetExportedKeys(call_options, &catalog, &schema, table)); } -TEST(TestFlightSqlClient, TestGetImported) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - FlightCallOptions call_options; - +TEST_F(TestFlightSqlClient, TestGetImported) { std::string catalog = "catalog"; std::string schema = "schema"; std::string table = "table"; @@ -253,11 +245,7 @@ TEST(TestFlightSqlClient, TestGetImported) { ASSERT_OK(sql_client.GetImportedKeys(call_options, &catalog, &schema, table)); } -TEST(TestFlightSqlClient, TestGetPrimary) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - FlightCallOptions call_options; - +TEST_F(TestFlightSqlClient, TestGetPrimary) { std::string catalog = "catalog"; std::string schema = "schema"; std::string table = "table"; @@ -273,11 +261,7 @@ TEST(TestFlightSqlClient, TestGetPrimary) { ASSERT_OK(sql_client.GetPrimaryKeys(call_options, &catalog, &schema, table)); } -TEST(TestFlightSqlClient, TestGetCrossReference) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - FlightCallOptions call_options; - +TEST_F(TestFlightSqlClient, TestGetCrossReference) { std::string pk_catalog = "pk_catalog"; std::string pk_schema = "pk_schema"; std::string pk_table = "pk_table"; @@ -300,11 +284,7 @@ TEST(TestFlightSqlClient, TestGetCrossReference) { &fk_catalog, &fk_schema, fk_table)); } -TEST(TestFlightSqlClient, TestExecute) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - FlightCallOptions call_options; - +TEST_F(TestFlightSqlClient, TestExecute) { std::string query = "query"; pb::sql::CommandStatementQuery command; @@ -316,11 +296,7 @@ TEST(TestFlightSqlClient, TestExecute) { ASSERT_OK(sql_client.Execute(call_options, query)); } -TEST(TestFlightSqlClient, TestPreparedStatementExecute) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - FlightCallOptions call_options; - +TEST_F(TestFlightSqlClient, TestPreparedStatementExecute) { const std::string query = "query"; ON_CALL(*client_mock, DoAction) @@ -342,18 +318,14 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecute) { EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); - ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client.Prepare(call_options, query)); + ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client->Prepare(call_options, query)); EXPECT_CALL(*client_mock, GetFlightInfo(_, _, _)); ASSERT_OK(prepared_statement->Execute()); } -TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - FlightCallOptions call_options; - +TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { const std::string query = "query"; ON_CALL(*client_mock, DoAction) @@ -399,7 +371,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); EXPECT_CALL(*client_mock, DoPut(_, _, _, _, _)); - ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client.Prepare(call_options, query)); + ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client->Prepare(call_options, query)); auto parameter_schema = prepared_statement->parameter_schema(); @@ -416,11 +388,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { ASSERT_OK(prepared_statement->Execute()); } -TEST(TestFlightSqlClient, TestExecuteUpdate) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - FlightCallOptions call_options; - +TEST_F(TestFlightSqlClient, TestExecuteUpdate) { std::string query = "query"; pb::sql::CommandStatementUpdate command; @@ -455,18 +423,16 @@ TEST(TestFlightSqlClient, TestExecuteUpdate) { std::unique_ptr reader; EXPECT_CALL(*client_mock, DoPut(Ref(call_options), descriptor, _, _, _)); - ASSERT_OK_AND_ASSIGN(auto num_rows, sql_client.ExecuteUpdate(call_options, query)); + ASSERT_OK_AND_ASSIGN(auto num_rows, sql_client->ExecuteUpdate(call_options, query)); ASSERT_EQ(num_rows, 100); } -TEST(TestFlightSqlClient, TestGetSqlInfo) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - - 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}; +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); @@ -474,19 +440,14 @@ TEST(TestFlightSqlClient, TestGetSqlInfo) { any.PackFrom(command); const FlightDescriptor& descriptor = FlightDescriptor::Command(any.SerializeAsString()); - FlightCallOptions call_options; EXPECT_CALL(*client_mock, 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) { - auto client_mock = std::make_shared(); - FlightSqlClient sql_client(client_mock); - + Func func, const std::shared_ptr* schema, FlightSqlClientT *sql_client) { const std::string query = "SELECT * FROM IRRELEVANT"; - const FlightCallOptions call_options; int64_t expected_rows = 100L; pb::sql::DoPutUpdateResult result; result.set_record_count(expected_rows); @@ -534,21 +495,21 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( EXPECT_CALL(*client_mock, DoPut(_, _, *schema, _, _)); } - ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client.Prepare(call_options, query)); + ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client->Prepare(call_options, query)); func(prepared_statement, *client_mock, schema, expected_rows); ASSERT_OK_AND_ASSIGN(auto rows, prepared_statement->ExecuteUpdate()); ASSERT_EQ(expected_rows, rows); } -TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateNoParameterBinding) { +TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateNoParameterBinding) { AssertTestPreparedStatementExecuteUpdateOk( [](const std::shared_ptr& prepared_statement, internal::FlightClientImpl& client_mock, const std::shared_ptr* schema, const int64_t& row_count) {}, - NULLPTR); + NULLPTR, sql_client); } -TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding) { +TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding) { const auto schema = arrow::schema( {arrow::field("field0", arrow::utf8()), arrow::field("field1", arrow::uint8())}); AssertTestPreparedStatementExecuteUpdateOk( @@ -565,7 +526,7 @@ TEST(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding RecordBatch::Make(*schema, row_count, {string_array, uint8_array}); ASSERT_OK(prepared_statement->SetParameters(recordBatch)); }, - &schema); + &schema, sql_client); } } // namespace sql diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index b1d7660a335..1017cdbda20 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -44,6 +44,12 @@ class TestFlightSqlServer : public ::testing::Environment { void SetUp() override { server = new TestServer("flight_sql_test_server"); server->Start(); + for(int i=0; i<100; i++) { + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + if(server->IsRunning()) { + break; + } + } ASSERT_TRUE(server->IsRunning()); std::stringstream ss; From 4abcd45a0e0706c9d8da1cbb79807971acbb0bf1 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Tue, 2 Nov 2021 15:18:00 -0300 Subject: [PATCH 180/237] fix: merge test client --- .../arrow/flight/flight_sql/client_test.cc | 50 +++++++++---------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index c61b247d5cd..1db77c546b3 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -92,16 +92,16 @@ std::shared_ptr client_mock; FlightCallOptions call_options; class TestFlightSqlClient : public ::testing::Test { -protected: - FlightSqlClient *sql_client; - void SetUp() override { - client_mock = std::make_shared(); - sql_client = new FlightSqlClient(client_mock); - } - - void TearDown() override { - free(sql_client); - } + protected: + FlightSqlClient *sql_client; + void SetUp() override { + client_mock = std::make_shared(); + sql_client = new FlightSqlClient(client_mock); + } + + void TearDown() override { + free(sql_client); + } }; class FlightMetadataReaderMock : public FlightMetadataReader { @@ -163,7 +163,7 @@ TEST_F(TestFlightSqlClient, TestGetCatalogs) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client.GetCatalogs(call_options)); + ASSERT_OK(sql_client->GetCatalogs(call_options)); } TEST_F(TestFlightSqlClient, TestGetSchemas) { @@ -177,7 +177,7 @@ TEST_F(TestFlightSqlClient, TestGetSchemas) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client.GetSchemas(call_options, &catalog, &schema_filter_pattern)); + ASSERT_OK(sql_client->GetSchemas(call_options, &catalog, &schema_filter_pattern)); } TEST_F(TestFlightSqlClient, TestGetTables) { @@ -199,7 +199,7 @@ TEST_F(TestFlightSqlClient, TestGetTables) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client.GetTables(call_options, &catalog, &schema_filter_pattern, + ASSERT_OK(sql_client->GetTables(call_options, &catalog, &schema_filter_pattern, &table_name_filter_pattern, include_schema, table_types)); } @@ -210,7 +210,7 @@ TEST_F(TestFlightSqlClient, TestGetTableTypes) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client.GetTableTypes(call_options)); + ASSERT_OK(sql_client->GetTableTypes(call_options)); } TEST_F(TestFlightSqlClient, TestGetExported) { @@ -226,7 +226,7 @@ TEST_F(TestFlightSqlClient, TestGetExported) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client.GetExportedKeys(call_options, &catalog, &schema, table)); + ASSERT_OK(sql_client->GetExportedKeys(call_options, &catalog, &schema, table)); } TEST_F(TestFlightSqlClient, TestGetImported) { @@ -242,7 +242,7 @@ TEST_F(TestFlightSqlClient, TestGetImported) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client.GetImportedKeys(call_options, &catalog, &schema, table)); + ASSERT_OK(sql_client->GetImportedKeys(call_options, &catalog, &schema, table)); } TEST_F(TestFlightSqlClient, TestGetPrimary) { @@ -258,7 +258,7 @@ TEST_F(TestFlightSqlClient, TestGetPrimary) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client.GetPrimaryKeys(call_options, &catalog, &schema, table)); + ASSERT_OK(sql_client->GetPrimaryKeys(call_options, &catalog, &schema, table)); } TEST_F(TestFlightSqlClient, TestGetCrossReference) { @@ -280,7 +280,7 @@ TEST_F(TestFlightSqlClient, TestGetCrossReference) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client.GetCrossReference(call_options, &pk_catalog, &pk_schema, pk_table, + ASSERT_OK(sql_client->GetCrossReference(call_options, &pk_catalog, &pk_schema, pk_table, &fk_catalog, &fk_schema, fk_table)); } @@ -293,7 +293,7 @@ TEST_F(TestFlightSqlClient, TestExecute) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client.Execute(call_options, query)); + ASSERT_OK(sql_client->Execute(call_options, query)); } TEST_F(TestFlightSqlClient, TestPreparedStatementExecute) { @@ -429,10 +429,10 @@ TEST_F(TestFlightSqlClient, TestExecuteUpdate) { } 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}; + + 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); @@ -441,12 +441,12 @@ TEST_F(TestFlightSqlClient, TestGetSqlInfo) { const FlightDescriptor& descriptor = FlightDescriptor::Command(any.SerializeAsString()); EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client.GetSqlInfo(call_options, sql_info)); + ASSERT_OK(sql_client->GetSqlInfo(call_options, sql_info)); } template inline void AssertTestPreparedStatementExecuteUpdateOk( - Func func, const std::shared_ptr* schema, FlightSqlClientT *sql_client) { + Func func, const std::shared_ptr* schema, FlightSqlClient *sql_client) { const std::string query = "SELECT * FROM IRRELEVANT"; int64_t expected_rows = 100L; pb::sql::DoPutUpdateResult result; From 75f67ad5e2cefde1cd0d6145a928066b6a9b5215 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Wed, 27 Oct 2021 11:02:19 -0300 Subject: [PATCH 181/237] Fix some issues on CMake and Server Test --- cpp/src/arrow/flight/flight_sql/server_test.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 1017cdbda20..4ce699f8dd0 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -353,7 +353,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) auto prepared_statement, sql_client->Prepare({}, "SELECT * FROM intTable WHERE keyName LIKE ?")); - auto parameter_schema = prepared_statement->parameter_schema(); + ASSERT_OK_AND_ASSIGN(auto parameter_schema, prepared_statement->GetParameterSchema()); const std::shared_ptr& expected_parameter_schema = arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); @@ -433,7 +433,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding sql_client->Prepare( {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', ?)")); - auto parameter_schema = prepared_statement->parameter_schema(); + ASSERT_OK_AND_ASSIGN(auto parameter_schema, prepared_statement->GetParameterSchema()); const std::shared_ptr& expected_parameter_schema = arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); From d480e77dd7c54a18823f2af3f676755d8f4e8785 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Thu, 28 Oct 2021 14:55:12 -0300 Subject: [PATCH 182/237] Made fixture to client_test.cc to avoid duplication --- cpp/src/arrow/flight/flight_sql/client_test.cc | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index 1db77c546b3..0fd733728a1 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -429,7 +429,6 @@ TEST_F(TestFlightSqlClient, TestExecuteUpdate) { } 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}; From 996c708e6835adad8468d5ece6ba5e5ca2693d4b Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Tue, 2 Nov 2021 16:28:08 -0300 Subject: [PATCH 183/237] Format test files --- cpp/src/arrow/flight/flight_sql/client_test.cc | 18 ++++++++---------- cpp/src/arrow/flight/flight_sql/server_test.cc | 14 +++++++------- 2 files changed, 15 insertions(+), 17 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index 0fd733728a1..cef97dc83ea 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -93,15 +93,13 @@ FlightCallOptions call_options; class TestFlightSqlClient : public ::testing::Test { protected: - FlightSqlClient *sql_client; + FlightSqlClient* sql_client; void SetUp() override { - client_mock = std::make_shared(); - sql_client = new FlightSqlClient(client_mock); + client_mock = std::make_shared(); + sql_client = new FlightSqlClient(client_mock); } - void TearDown() override { - free(sql_client); - } + void TearDown() override { free(sql_client); } }; class FlightMetadataReaderMock : public FlightMetadataReader { @@ -200,8 +198,8 @@ TEST_F(TestFlightSqlClient, TestGetTables) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); ASSERT_OK(sql_client->GetTables(call_options, &catalog, &schema_filter_pattern, - &table_name_filter_pattern, include_schema, - table_types)); + &table_name_filter_pattern, include_schema, + table_types)); } TEST_F(TestFlightSqlClient, TestGetTableTypes) { @@ -281,7 +279,7 @@ TEST_F(TestFlightSqlClient, TestGetCrossReference) { EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); ASSERT_OK(sql_client->GetCrossReference(call_options, &pk_catalog, &pk_schema, pk_table, - &fk_catalog, &fk_schema, fk_table)); + &fk_catalog, &fk_schema, fk_table)); } TEST_F(TestFlightSqlClient, TestExecute) { @@ -445,7 +443,7 @@ TEST_F(TestFlightSqlClient, TestGetSqlInfo) { template inline void AssertTestPreparedStatementExecuteUpdateOk( - Func func, const std::shared_ptr* schema, FlightSqlClient *sql_client) { + Func func, const std::shared_ptr* schema, FlightSqlClient* sql_client) { const std::string query = "SELECT * FROM IRRELEVANT"; int64_t expected_rows = 100L; pb::sql::DoPutUpdateResult result; diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 4ce699f8dd0..05d1c23a726 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -44,11 +44,11 @@ class TestFlightSqlServer : public ::testing::Environment { void SetUp() override { server = new TestServer("flight_sql_test_server"); server->Start(); - for(int i=0; i<100; i++) { - std::this_thread::sleep_for(std::chrono::milliseconds(10)); - if(server->IsRunning()) { - break; - } + for (int i = 0; i < 100; i++) { + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + if (server->IsRunning()) { + break; + } } ASSERT_TRUE(server->IsRunning()); @@ -353,7 +353,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) auto prepared_statement, sql_client->Prepare({}, "SELECT * FROM intTable WHERE keyName LIKE ?")); - ASSERT_OK_AND_ASSIGN(auto parameter_schema, prepared_statement->GetParameterSchema()); + auto parameter_schema = prepared_statement->parameter_schema(); const std::shared_ptr& expected_parameter_schema = arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); @@ -433,7 +433,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding sql_client->Prepare( {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', ?)")); - ASSERT_OK_AND_ASSIGN(auto parameter_schema, prepared_statement->GetParameterSchema()); + auto parameter_schema = prepared_statement->parameter_schema(); const std::shared_ptr& expected_parameter_schema = arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); From 89b4f17cf9aac989f9aa09fa6a0d2113532d9a9d Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Wed, 3 Nov 2021 14:04:58 -0300 Subject: [PATCH 184/237] fix: correct issues on tests --- cpp/src/arrow/flight/flight_sql/client_test.cc | 9 ++++----- cpp/src/arrow/flight/flight_sql/server_test.cc | 7 ++++++- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index cef97dc83ea..f4b33e9ee86 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -93,13 +93,11 @@ FlightCallOptions call_options; class TestFlightSqlClient : public ::testing::Test { protected: - FlightSqlClient* sql_client; + std::unique_ptr sql_client; void SetUp() override { client_mock = std::make_shared(); - sql_client = new FlightSqlClient(client_mock); + sql_client.reset(new FlightSqlClient(client_mock)); } - - void TearDown() override { free(sql_client); } }; class FlightMetadataReaderMock : public FlightMetadataReader { @@ -443,7 +441,8 @@ TEST_F(TestFlightSqlClient, TestGetSqlInfo) { template inline void AssertTestPreparedStatementExecuteUpdateOk( - Func func, const std::shared_ptr* schema, FlightSqlClient* sql_client) { + Func func, const std::shared_ptr* schema, + std::unique_ptr& sql_client) { const std::string query = "SELECT * FROM IRRELEVANT"; int64_t expected_rows = 100L; pb::sql::DoPutUpdateResult result; diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 05d1c23a726..8ba1de761c2 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -65,7 +65,12 @@ class TestFlightSqlServer : public ::testing::Environment { } void TearDown() override { - server->Stop(); + for (int i = 0; i < 100; i++) { + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + if (server->Stop()) { + break; + } + } delete server; delete sql_client; From 24343e24df274eb1566971ab139f7ac33dc5fb6a Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Wed, 3 Nov 2021 15:42:56 -0300 Subject: [PATCH 185/237] Fix memory leak with mock --- cpp/src/arrow/flight/flight_sql/client_test.cc | 2 ++ cpp/src/arrow/flight/flight_sql/server_test.cc | 10 +++++----- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index f4b33e9ee86..1b2aeb5d9eb 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -98,6 +98,8 @@ class TestFlightSqlClient : public ::testing::Test { client_mock = std::make_shared(); sql_client.reset(new FlightSqlClient(client_mock)); } + + void TearDown() override { client_mock.reset(); } }; class FlightMetadataReaderMock : public FlightMetadataReader { diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 8ba1de761c2..4b5eb8dc644 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -65,12 +65,12 @@ class TestFlightSqlServer : public ::testing::Environment { } void TearDown() override { - for (int i = 0; i < 100; i++) { - std::this_thread::sleep_for(std::chrono::milliseconds(10)); - if (server->Stop()) { - break; - } + for (int i = 0; i < 100; i++) { + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + if (server->Stop()) { + break; } + } delete server; delete sql_client; From 70bda0c5c117121787922987b6687749c73a2ea0 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Wed, 3 Nov 2021 16:01:06 -0300 Subject: [PATCH 186/237] Fix test_server --- cpp/src/arrow/flight/flight_sql/server_test.cc | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 4b5eb8dc644..10976e9ed9c 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -66,12 +66,11 @@ class TestFlightSqlServer : public ::testing::Environment { void TearDown() override { for (int i = 0; i < 100; i++) { - std::this_thread::sleep_for(std::chrono::milliseconds(10)); - if (server->Stop()) { + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + if (!(server->IsRunning())) { break; } } - delete server; delete sql_client; } From 698645d127de4647e72cc93236f4700fc0a7a6ea Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Wed, 3 Nov 2021 17:26:24 -0300 Subject: [PATCH 187/237] fix server stop on serve rt --- cpp/src/arrow/flight/flight_sql/server_test.cc | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 10976e9ed9c..42ca093adb7 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -65,12 +65,7 @@ class TestFlightSqlServer : public ::testing::Environment { } void TearDown() override { - for (int i = 0; i < 100; i++) { - std::this_thread::sleep_for(std::chrono::milliseconds(10)); - if (!(server->IsRunning())) { - break; - } - } + server->Stop(); delete server; delete sql_client; } From 46569759a234cf62a1897bdd2ae12687faba852c Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Wed, 3 Nov 2021 18:23:23 -0300 Subject: [PATCH 188/237] Change server to unique pointer on server tests --- cpp/src/arrow/flight/flight_sql/server_test.cc | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 42ca093adb7..51ace2ec20c 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -36,13 +36,13 @@ namespace arrow { namespace flight { namespace sql { -TestServer* server; +std::unique_ptr server; FlightSqlClient* sql_client; class TestFlightSqlServer : public ::testing::Environment { protected: void SetUp() override { - server = new TestServer("flight_sql_test_server"); + server.reset(new TestServer("flight_sql_test_server")); server->Start(); for (int i = 0; i < 100; i++) { std::this_thread::sleep_for(std::chrono::milliseconds(10)); @@ -66,7 +66,6 @@ class TestFlightSqlServer : public ::testing::Environment { void TearDown() override { server->Stop(); - delete server; delete sql_client; } }; From 2483ff4cfd7674a2ad5f6a432569419bcbf179d9 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Thu, 4 Nov 2021 11:49:11 -0300 Subject: [PATCH 189/237] Change sql_client to unique_pointer on server test --- cpp/src/arrow/flight/flight_sql/server_test.cc | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 51ace2ec20c..b35e8af76f2 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -37,7 +37,7 @@ namespace flight { namespace sql { std::unique_ptr server; -FlightSqlClient* sql_client; +std::unique_ptr sql_client; class TestFlightSqlServer : public ::testing::Environment { protected: @@ -61,12 +61,11 @@ class TestFlightSqlServer : public ::testing::Environment { ASSERT_OK(Location::Parse(uri, &location)); ASSERT_OK(FlightClient::Connect(location, &client)); - sql_client = new FlightSqlClient(std::move(client)); + sql_client.reset(new FlightSqlClient(std::move(client))); } void TearDown() override { server->Stop(); - delete sql_client; } }; From 9135703c25e4f54bd3d487676ce62739fbb3e52e Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Thu, 4 Nov 2021 12:41:09 -0300 Subject: [PATCH 190/237] server.reset() and sql_client.reset() on server test TearDown --- cpp/src/arrow/flight/flight_sql/server_test.cc | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index b35e8af76f2..de667ba8703 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -66,6 +66,8 @@ class TestFlightSqlServer : public ::testing::Environment { void TearDown() override { server->Stop(); + server.reset(); + sql_client.reset(); } }; From f604d468f9ba885a51f738c93c055d8f8d064f19 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Thu, 4 Nov 2021 13:00:14 -0300 Subject: [PATCH 191/237] Fix sql_cliente close before server on server test --- cpp/src/arrow/flight/flight_sql/server_test.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index de667ba8703..7001693862c 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -66,8 +66,8 @@ class TestFlightSqlServer : public ::testing::Environment { void TearDown() override { server->Stop(); - server.reset(); sql_client.reset(); + server.reset(); } }; From e2e57ae39876e1978138ca2b3d66ea9ecacaf7fc Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Thu, 4 Nov 2021 13:04:00 -0300 Subject: [PATCH 192/237] Fix sql_client close before server stop server test --- cpp/src/arrow/flight/flight_sql/server_test.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 7001693862c..9e9fd9f72b2 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -65,8 +65,8 @@ class TestFlightSqlServer : public ::testing::Environment { } void TearDown() override { - server->Stop(); sql_client.reset(); + server->Stop(); server.reset(); } }; From 0b32b272ac1bd9b16d1bd4a7d2521db9780ffc30 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Mon, 8 Nov 2021 15:03:58 -0300 Subject: [PATCH 193/237] implement FindSQLite3Alt on cmake modules --- cpp/cmake_modules/FindSQLite3Alt.cmake | 42 +++++++++++++++++++ .../arrow/flight/flight_sql/CMakeLists.txt | 3 +- 2 files changed, 44 insertions(+), 1 deletion(-) create mode 100644 cpp/cmake_modules/FindSQLite3Alt.cmake diff --git a/cpp/cmake_modules/FindSQLite3Alt.cmake b/cpp/cmake_modules/FindSQLite3Alt.cmake new file mode 100644 index 00000000000..559931b1b7d --- /dev/null +++ b/cpp/cmake_modules/FindSQLite3Alt.cmake @@ -0,0 +1,42 @@ +# 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 FAULT_MSG SQLite3_LIBRARY SQLite3_INCLUDE_DIR) + +mark_as_advanced(SQLite3_LIBRARY SQLite3_INCLUDE_DIR) + +if(SQLite3_FOUND) + SET(SQLite3_INCLUDE_DIRS ${SQLite3_INCLUDE_DIR}) + SET(SQLite3_LIBRARIES ${SQLite3_LIBRARY}) +endif() diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index eeaad62e363..69417f4a841 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -99,7 +99,8 @@ add_arrow_test(flight_sql_server_test # Build test server for unit tests if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) - find_package(SQLite3) + find_package(SQLite3Alt REQUIRED) + include_directories(${SQLite3_INCLUDE_DIRS}) add_executable(flight_sql_test_server server.cc From 71f877df2e34b10e5c83e3540aca164ec0d46873 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Mon, 8 Nov 2021 16:13:34 -0300 Subject: [PATCH 194/237] Remove wrong line on CmakeLists --- cpp/src/arrow/flight/flight_sql/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index 69417f4a841..425d898d748 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -100,7 +100,6 @@ add_arrow_test(flight_sql_server_test # Build test server for unit tests if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) find_package(SQLite3Alt REQUIRED) - include_directories(${SQLite3_INCLUDE_DIRS}) add_executable(flight_sql_test_server server.cc From 093b539df8f16e6e319869b7acbdae7789e91e47 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Tue, 9 Nov 2021 16:03:32 -0300 Subject: [PATCH 195/237] Fix protobuf version on flight-core --- java/flight/flight-core/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 From d40ef9994e532538615d4ca16a908db1dd8df631 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Tue, 9 Nov 2021 17:57:42 -0300 Subject: [PATCH 196/237] Reestructure FlightSqlClient to use virtual methods (#195) --- .../arrow/flight/flight_sql/CMakeLists.txt | 16 +- cpp/src/arrow/flight/flight_sql/client.cc | 62 +++----- cpp/src/arrow/flight/flight_sql/client.h | 41 +++-- .../flight/flight_sql/client_internal.cc | 94 ----------- .../arrow/flight/flight_sql/client_internal.h | 61 -------- .../arrow/flight/flight_sql/client_test.cc | 148 +++++++----------- 6 files changed, 115 insertions(+), 307 deletions(-) delete mode 100644 cpp/src/arrow/flight/flight_sql/client_internal.cc delete mode 100644 cpp/src/arrow/flight/flight_sql/client_internal.h diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index 425d898d748..fd28559a91b 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -36,7 +36,7 @@ set_source_files_properties(${FLIGHT_SQL_GENERATED_PROTO_FILES} PROPERTIES GENER add_custom_target(flight_sql_protobuf_gen ALL DEPENDS ${FLIGHT_SQL_GENERATED_PROTO_FILES}) -set(ARROW_FLIGHT_SQL_SRCS server.cc client.cc client_internal.cc) +set(ARROW_FLIGHT_SQL_SRCS server.cc client.cc) add_arrow_lib(arrow_flight_sql CMAKE_PACKAGE_NAME @@ -77,20 +77,10 @@ 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}) -add_arrow_test(flight_sql_client_test +add_arrow_test(flight_sql_test SOURCES - "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" client_test.cc - client.cc - STATIC_LINK_LIBS - ${ARROW_FLIGHT_TEST_LINK_LIBS} - LABELS - "arrow_flight_sql") - -add_arrow_test(flight_sql_server_test - SOURCES server_test.cc - client_internal.cc "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" STATIC_LINK_LIBS ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} @@ -114,6 +104,6 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) ${SQLite3_LIBRARIES}) if(ARROW_BUILD_TESTS) - add_dependencies(arrow-flight-sql-server-test flight_sql_test_server) + add_dependencies(arrow-flight-sql-test flight_sql_test_server) endif() endif() diff --git a/cpp/src/arrow/flight/flight_sql/client.cc b/cpp/src/arrow/flight/flight_sql/client.cc index dfc77e1698d..77891e55536 100644 --- a/cpp/src/arrow/flight/flight_sql/client.cc +++ b/cpp/src/arrow/flight/flight_sql/client.cc @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -33,18 +32,14 @@ namespace arrow { namespace flight { namespace sql { -FlightSqlClient::FlightSqlClient(std::shared_ptr client) +FlightSqlClient::FlightSqlClient(std::shared_ptr client) : impl_(std::move(client)) {} -FlightSqlClient::FlightSqlClient(std::unique_ptr client) - : impl_(internal::FlightClientImpl_Create(std::move(client))) {} - -PreparedStatement::PreparedStatement(std::shared_ptr client, - std::string handle, +PreparedStatement::PreparedStatement(FlightSqlClient& client, std::string handle, std::shared_ptr dataset_schema, std::shared_ptr parameter_schema, FlightCallOptions options) - : client_(std::move(client)), + : client_(client), options_(std::move(options)), handle_(std::move(handle)), dataset_schema_(std::move(dataset_schema)), @@ -72,13 +67,12 @@ inline FlightDescriptor GetFlightDescriptorForCommand( } arrow::Result> GetFlightInfoForCommand( - internal::FlightClientImpl& client, const FlightCallOptions& options, + FlightSqlClient& client, const FlightCallOptions& options, const google::protobuf::Message& command) { const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); std::unique_ptr flight_info; - ARROW_RETURN_NOT_OK(internal::FlightClientImpl_GetFlightInfo(client, options, - descriptor, &flight_info)); + ARROW_RETURN_NOT_OK(client.GetFlightInfo(options, descriptor, &flight_info)); return std::move(flight_info); } @@ -88,7 +82,7 @@ arrow::Result> FlightSqlClient::Execute( pb::sql::CommandStatementQuery command; command.set_query(query); - return GetFlightInfoForCommand(*impl_, options, command); + return GetFlightInfoForCommand(*this, options, command); } arrow::Result FlightSqlClient::ExecuteUpdate(const FlightCallOptions& options, @@ -101,8 +95,7 @@ arrow::Result FlightSqlClient::ExecuteUpdate(const FlightCallOptions& o std::unique_ptr writer; std::unique_ptr reader; - ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoPut(*impl_, options, descriptor, - NULLPTR, &writer, &reader)); + ARROW_RETURN_NOT_OK(DoPut(options, descriptor, NULLPTR, &writer, &reader)); std::shared_ptr metadata; @@ -122,7 +115,7 @@ arrow::Result> FlightSqlClient::GetCatalogs( const FlightCallOptions& options) { pb::sql::CommandGetCatalogs command; - return GetFlightInfoForCommand(*impl_, options, command); + return GetFlightInfoForCommand(*this, options, command); } arrow::Result> FlightSqlClient::GetSchemas( @@ -136,7 +129,7 @@ arrow::Result> FlightSqlClient::GetSchemas( command.set_schema_filter_pattern(*schema_filter_pattern); } - return GetFlightInfoForCommand(*impl_, options, command); + return GetFlightInfoForCommand(*this, options, command); } arrow::Result> FlightSqlClient::GetTables( @@ -163,7 +156,7 @@ arrow::Result> FlightSqlClient::GetTables( command.add_table_types(table_type); } - return GetFlightInfoForCommand(*impl_, options, command); + return GetFlightInfoForCommand(*this, options, command); } arrow::Result> FlightSqlClient::GetPrimaryKeys( @@ -181,7 +174,7 @@ arrow::Result> FlightSqlClient::GetPrimaryKeys( command.set_table(table); - return GetFlightInfoForCommand(*impl_, options, command); + return GetFlightInfoForCommand(*this, options, command); } arrow::Result> FlightSqlClient::GetExportedKeys( @@ -199,7 +192,7 @@ arrow::Result> FlightSqlClient::GetExportedKeys( command.set_table(table); - return GetFlightInfoForCommand(*impl_, options, command); + return GetFlightInfoForCommand(*this, options, command); } arrow::Result> FlightSqlClient::GetImportedKeys( @@ -217,7 +210,7 @@ arrow::Result> FlightSqlClient::GetImportedKeys( command.set_table(table); - return GetFlightInfoForCommand(*impl_, options, command); + return GetFlightInfoForCommand(*this, options, command); } arrow::Result> FlightSqlClient::GetCrossReference( @@ -243,20 +236,20 @@ arrow::Result> FlightSqlClient::GetCrossReference( } command.set_fk_table(fk_table); - return GetFlightInfoForCommand(*impl_, options, command); + return GetFlightInfoForCommand(*this, options, command); } arrow::Result> FlightSqlClient::GetTableTypes( const FlightCallOptions& options) { pb::sql::CommandGetTableTypes command; - return GetFlightInfoForCommand(*impl_, options, command); + return GetFlightInfoForCommand(*this, options, command); } arrow::Result> FlightSqlClient::DoGet( const FlightCallOptions& options, const Ticket& ticket) { std::unique_ptr stream; - ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoGet(*impl_, options, ticket, &stream)); + ARROW_RETURN_NOT_OK(DoGet(options, ticket, &stream)); return std::move(stream); } @@ -274,8 +267,7 @@ arrow::Result> FlightSqlClient::Prepare( std::unique_ptr results; - ARROW_RETURN_NOT_OK( - internal::FlightClientImpl_DoAction(*impl_, options, action, &results)); + ARROW_RETURN_NOT_OK(DoAction(options, action, &results)); std::unique_ptr result; ARROW_RETURN_NOT_OK(results->Next(&result)); @@ -314,7 +306,7 @@ arrow::Result> FlightSqlClient::Prepare( } auto handle = prepared_statement_result.prepared_statement_handle(); - return std::make_shared(impl_, handle, dataset_schema, + return std::make_shared(*this, handle, dataset_schema, parameter_schema, options); } @@ -336,8 +328,8 @@ arrow::Result> PreparedStatement::Execute() { if (parameter_binding_ && parameter_binding_->num_rows() > 0) { std::unique_ptr writer; std::unique_ptr reader; - ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoPut( - *client_, options_, descriptor, parameter_binding_->schema(), &writer, &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()); @@ -347,8 +339,7 @@ arrow::Result> PreparedStatement::Execute() { } std::unique_ptr info; - ARROW_RETURN_NOT_OK( - internal::FlightClientImpl_GetFlightInfo(*client_, options_, descriptor, &info)); + ARROW_RETURN_NOT_OK(client_.GetFlightInfo(options_, descriptor, &info)); return std::move(info); } @@ -365,13 +356,12 @@ arrow::Result PreparedStatement::ExecuteUpdate() { std::unique_ptr reader; if (parameter_binding_ && parameter_binding_->num_rows() > 0) { - ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoPut( - *client_, options_, descriptor, parameter_binding_->schema(), &writer, &reader)); + 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(internal::FlightClientImpl_DoPut(*client_, options_, descriptor, - schema, &writer, &reader)); + 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)); @@ -422,7 +412,7 @@ Status PreparedStatement::Close() { std::unique_ptr results; - ARROW_RETURN_NOT_OK(FlightClientImpl_DoAction(*client_, options_, action, &results)); + ARROW_RETURN_NOT_OK(client_.DoAction(options_, action, &results)); is_closed_ = true; @@ -434,7 +424,7 @@ arrow::Result> FlightSqlClient::GetSqlInfo( pb::sql::CommandGetSqlInfo command; for (const int& info : sql_info) command.add_info(info); - return GetFlightInfoForCommand(*impl_, options, command); + return GetFlightInfoForCommand(*this, options, command); } } // namespace sql diff --git a/cpp/src/arrow/flight/flight_sql/client.h b/cpp/src/arrow/flight/flight_sql/client.h index 20c8dbd30e6..4b035778495 100644 --- a/cpp/src/arrow/flight/flight_sql/client.h +++ b/cpp/src/arrow/flight/flight_sql/client.h @@ -29,19 +29,17 @@ namespace arrow { namespace flight { namespace sql { -namespace internal { -class FlightClientImpl; -} - class PreparedStatement; /// \brief Flight client with Flight SQL semantics. class ARROW_EXPORT FlightSqlClient { + friend class PreparedStatement; + private: - std::shared_ptr impl_; + std::shared_ptr impl_; public: - explicit FlightSqlClient(std::shared_ptr client); + explicit FlightSqlClient(std::shared_ptr client); ~FlightSqlClient(); @@ -164,11 +162,34 @@ class ARROW_EXPORT FlightSqlClient { arrow::Result> Prepare( const FlightCallOptions& options, const std::string& query); - explicit FlightSqlClient(std::unique_ptr client); + virtual Status GetFlightInfo(const FlightCallOptions& options, + const FlightDescriptor& descriptor, + std::unique_ptr* info) { + return impl_->GetFlightInfo(options, descriptor, 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); + } }; class PreparedStatement { - std::shared_ptr client_; + FlightSqlClient& client_; FlightCallOptions options_; std::string handle_; std::shared_ptr dataset_schema_; @@ -183,8 +204,8 @@ class PreparedStatement { /// \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(std::shared_ptr client, - std::string handle, std::shared_ptr dataset_schema, + 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. diff --git a/cpp/src/arrow/flight/flight_sql/client_internal.cc b/cpp/src/arrow/flight/flight_sql/client_internal.cc deleted file mode 100644 index dcb645c6875..00000000000 --- a/cpp/src/arrow/flight/flight_sql/client_internal.cc +++ /dev/null @@ -1,94 +0,0 @@ -// 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 - -namespace arrow { -namespace flight { -namespace sql { -namespace internal { -class FlightClientImpl { - public: - explicit FlightClientImpl(std::unique_ptr client) - : client_(std::move(client)) {} - - ~FlightClientImpl() = default; - - Status GetFlightInfo(const FlightCallOptions& options, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { - return client_->GetFlightInfo(options, descriptor, info); - } - - Status DoPut(const FlightCallOptions& options, const FlightDescriptor& descriptor, - const std::shared_ptr& schema, - std::unique_ptr* stream, - std::unique_ptr* reader) { - return client_->DoPut(options, descriptor, schema, stream, reader); - } - - Status DoGet(const FlightCallOptions& options, const Ticket& ticket, - std::unique_ptr* stream) { - return client_->DoGet(options, ticket, stream); - } - - Status DoAction(const FlightCallOptions& options, const Action& action, - std::unique_ptr* results) { - return client_->DoAction(options, action, results); - } - - private: - std::unique_ptr client_; -}; - -Status FlightClientImpl_GetFlightInfo(FlightClientImpl& client, - const FlightCallOptions& options, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { - return client.GetFlightInfo(options, descriptor, info); -} - -Status FlightClientImpl_DoPut(FlightClientImpl& client, const FlightCallOptions& options, - const FlightDescriptor& descriptor, - const std::shared_ptr& schema, - std::unique_ptr* stream, - std::unique_ptr* reader) { - return client.DoPut(options, descriptor, schema, stream, reader); -} - -Status FlightClientImpl_DoGet(FlightClientImpl& client, const FlightCallOptions& options, - const Ticket& ticket, - std::unique_ptr* stream) { - return client.DoGet(options, ticket, stream); -} - -Status FlightClientImpl_DoAction(FlightClientImpl& client, - const FlightCallOptions& options, const Action& action, - std::unique_ptr* results) { - return client.DoAction(options, action, results); -} - -std::shared_ptr FlightClientImpl_Create( - std::unique_ptr client) { - return std::make_shared(std::move(client)); -} - -} // namespace internal -} // namespace sql -} // namespace flight -} // namespace arrow diff --git a/cpp/src/arrow/flight/flight_sql/client_internal.h b/cpp/src/arrow/flight/flight_sql/client_internal.h deleted file mode 100644 index 23eba399b9d..00000000000 --- a/cpp/src/arrow/flight/flight_sql/client_internal.h +++ /dev/null @@ -1,61 +0,0 @@ -// 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 - -namespace arrow { -namespace flight { - -class FlightClient; -class FlightCallOptions; -class FlightStreamWriter; -class FlightMetadataReader; -class FlightStreamReader; - -namespace sql { -namespace internal { - -class FlightClientImpl; - -Status FlightClientImpl_GetFlightInfo(FlightClientImpl& client, - const FlightCallOptions& options, - const FlightDescriptor& descriptor, - std::unique_ptr* info); - -Status FlightClientImpl_DoPut(FlightClientImpl& client, const FlightCallOptions& options, - const FlightDescriptor& descriptor, - const std::shared_ptr& schema, - std::unique_ptr* stream, - std::unique_ptr* reader); - -Status FlightClientImpl_DoGet(FlightClientImpl& client, const FlightCallOptions& options, - const Ticket& ticket, - std::unique_ptr* stream); - -Status FlightClientImpl_DoAction(FlightClientImpl& client, - const FlightCallOptions& options, const Action& action, - std::unique_ptr* results); - -std::shared_ptr FlightClientImpl_Create( - std::unique_ptr client); - -} // namespace internal -} // namespace sql -} // namespace flight -} // namespace arrow diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index 1b2aeb5d9eb..2b7c6025125 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -33,10 +33,11 @@ namespace arrow { namespace flight { namespace sql { -namespace internal { -class FlightClientImpl { +class FlightSqlClientMock : public FlightSqlClient { public: - ~FlightClientImpl() = default; + FlightSqlClientMock() : FlightSqlClient(nullptr) {} + + ~FlightSqlClientMock() = default; MOCK_METHOD(Status, GetFlightInfo, (const FlightCallOptions&, const FlightDescriptor&, @@ -54,52 +55,14 @@ class FlightClientImpl { std::unique_ptr* results)); }; -Status FlightClientImpl_GetFlightInfo(FlightClientImpl& client, - const FlightCallOptions& options, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { - return client.GetFlightInfo(options, descriptor, info); -} - -Status FlightClientImpl_DoPut(FlightClientImpl& client, const FlightCallOptions& options, - const FlightDescriptor& descriptor, - const std::shared_ptr& schema, - std::unique_ptr* stream, - std::unique_ptr* reader) { - return client.DoPut(options, descriptor, schema, stream, reader); -} - -Status FlightClientImpl_DoGet(FlightClientImpl& client, const FlightCallOptions& options, - const Ticket& ticket, - std::unique_ptr* stream) { - return client.DoGet(options, ticket, stream); -} - -Status FlightClientImpl_DoAction(FlightClientImpl& client, - const FlightCallOptions& options, const Action& action, - std::unique_ptr* results) { - return client.DoAction(options, action, results); -} - -std::shared_ptr FlightClientImpl_Create( - std::unique_ptr client) { - return std::make_shared(); -} - -} // namespace internal - -std::shared_ptr client_mock; -FlightCallOptions call_options; - class TestFlightSqlClient : public ::testing::Test { protected: - std::unique_ptr sql_client; - void SetUp() override { - client_mock = std::make_shared(); - sql_client.reset(new FlightSqlClient(client_mock)); - } + FlightSqlClientMock sql_client; + FlightCallOptions call_options; + + void SetUp() override {} - void TearDown() override { client_mock.reset(); } + void TearDown() override {} }; class FlightMetadataReaderMock : public FlightMetadataReader { @@ -159,9 +122,9 @@ TEST_F(TestFlightSqlClient, TestGetCatalogs) { pb::sql::CommandGetCatalogs command; FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); + EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client->GetCatalogs(call_options)); + ASSERT_OK(sql_client.GetCatalogs(call_options)); } TEST_F(TestFlightSqlClient, TestGetSchemas) { @@ -173,9 +136,9 @@ TEST_F(TestFlightSqlClient, TestGetSchemas) { command.set_schema_filter_pattern(schema_filter_pattern); FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); + EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client->GetSchemas(call_options, &catalog, &schema_filter_pattern)); + ASSERT_OK(sql_client.GetSchemas(call_options, &catalog, &schema_filter_pattern)); } TEST_F(TestFlightSqlClient, TestGetTables) { @@ -195,20 +158,20 @@ TEST_F(TestFlightSqlClient, TestGetTables) { } FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); + 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)); + 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); - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); + EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client->GetTableTypes(call_options)); + ASSERT_OK(sql_client.GetTableTypes(call_options)); } TEST_F(TestFlightSqlClient, TestGetExported) { @@ -222,9 +185,9 @@ TEST_F(TestFlightSqlClient, TestGetExported) { command.set_table(table); FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); + EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client->GetExportedKeys(call_options, &catalog, &schema, table)); + ASSERT_OK(sql_client.GetExportedKeys(call_options, &catalog, &schema, table)); } TEST_F(TestFlightSqlClient, TestGetImported) { @@ -238,9 +201,9 @@ TEST_F(TestFlightSqlClient, TestGetImported) { command.set_table(table); FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); + EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client->GetImportedKeys(call_options, &catalog, &schema, table)); + ASSERT_OK(sql_client.GetImportedKeys(call_options, &catalog, &schema, table)); } TEST_F(TestFlightSqlClient, TestGetPrimary) { @@ -254,9 +217,9 @@ TEST_F(TestFlightSqlClient, TestGetPrimary) { command.set_table(table); FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); + EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client->GetPrimaryKeys(call_options, &catalog, &schema, table)); + ASSERT_OK(sql_client.GetPrimaryKeys(call_options, &catalog, &schema, table)); } TEST_F(TestFlightSqlClient, TestGetCrossReference) { @@ -276,10 +239,10 @@ TEST_F(TestFlightSqlClient, TestGetCrossReference) { command.set_fk_table(fk_table); FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); + EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client->GetCrossReference(call_options, &pk_catalog, &pk_schema, pk_table, - &fk_catalog, &fk_schema, fk_table)); + ASSERT_OK(sql_client.GetCrossReference(call_options, &pk_catalog, &pk_schema, pk_table, + &fk_catalog, &fk_schema, fk_table)); } TEST_F(TestFlightSqlClient, TestExecute) { @@ -289,15 +252,15 @@ TEST_F(TestFlightSqlClient, TestExecute) { command.set_query(query); FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); + EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client->Execute(call_options, query)); + ASSERT_OK(sql_client.Execute(call_options, query)); } TEST_F(TestFlightSqlClient, TestPreparedStatementExecute) { const std::string query = "query"; - ON_CALL(*client_mock, DoAction) + ON_CALL(sql_client, DoAction) .WillByDefault([](const FlightCallOptions& options, const Action& action, std::unique_ptr* results) { google::protobuf::Any command; @@ -314,11 +277,11 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecute) { return Status::OK(); }); - EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); + EXPECT_CALL(sql_client, DoAction(_, _, _)).Times(2); - ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client->Prepare(call_options, query)); + ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client.Prepare(call_options, query)); - EXPECT_CALL(*client_mock, GetFlightInfo(_, _, _)); + EXPECT_CALL(sql_client, GetFlightInfo(_, _, _)); ASSERT_OK(prepared_statement->Execute()); } @@ -326,7 +289,7 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecute) { TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { const std::string query = "query"; - ON_CALL(*client_mock, DoAction) + ON_CALL(sql_client, DoAction) .WillByDefault([](const FlightCallOptions& options, const Action& action, std::unique_ptr* results) { google::protobuf::Any command; @@ -354,7 +317,7 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { }); std::shared_ptr buffer_ptr; - ON_CALL(*client_mock, DoPut) + ON_CALL(sql_client, DoPut) .WillByDefault([&buffer_ptr](const FlightCallOptions& options, const FlightDescriptor& descriptor1, const std::shared_ptr& schema, @@ -366,10 +329,10 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { return Status::OK(); }); - EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); - EXPECT_CALL(*client_mock, DoPut(_, _, _, _, _)); + 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)); + ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client.Prepare(call_options, query)); auto parameter_schema = prepared_statement->parameter_schema(); @@ -381,7 +344,7 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { result = arrow::RecordBatch::Make(parameter_schema, 1, {int_array}); ASSERT_OK(prepared_statement->SetParameters(result)); - EXPECT_CALL(*client_mock, GetFlightInfo(_, _, _)); + EXPECT_CALL(sql_client, GetFlightInfo(_, _, _)); ASSERT_OK(prepared_statement->Execute()); } @@ -405,7 +368,7 @@ TEST_F(TestFlightSqlClient, TestExecuteUpdate) { auto buffer_ptr = std::make_shared( reinterpret_cast(string.data()), doPutUpdateResult.ByteSizeLong()); - ON_CALL(*client_mock, DoPut) + ON_CALL(sql_client, DoPut) .WillByDefault([&buffer_ptr](const FlightCallOptions& options, const FlightDescriptor& descriptor1, const std::shared_ptr& schema, @@ -419,9 +382,9 @@ TEST_F(TestFlightSqlClient, TestExecuteUpdate) { std::unique_ptr flight_info; std::unique_ptr writer; std::unique_ptr reader; - EXPECT_CALL(*client_mock, DoPut(Ref(call_options), descriptor, _, _, _)); + EXPECT_CALL(sql_client, DoPut(Ref(call_options), descriptor, _, _, _)); - ASSERT_OK_AND_ASSIGN(auto num_rows, sql_client->ExecuteUpdate(call_options, query)); + ASSERT_OK_AND_ASSIGN(auto num_rows, sql_client.ExecuteUpdate(call_options, query)); ASSERT_EQ(num_rows, 100); } @@ -437,20 +400,19 @@ TEST_F(TestFlightSqlClient, TestGetSqlInfo) { any.PackFrom(command); const FlightDescriptor& descriptor = FlightDescriptor::Command(any.SerializeAsString()); - EXPECT_CALL(*client_mock, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client->GetSqlInfo(call_options, sql_info)); + 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, - std::unique_ptr& sql_client) { + 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(*client_mock, DoAction) + ON_CALL(sql_client, DoAction) .WillByDefault([&query, &schema](const FlightCallOptions& options, const Action& action, std::unique_ptr* results) { @@ -474,10 +436,10 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( return Status::OK(); }); - EXPECT_CALL(*client_mock, DoAction(_, _, _)).Times(2); + EXPECT_CALL(sql_client, DoAction(_, _, _)).Times(2); auto buffer = Buffer::FromString(result.SerializeAsString()); - ON_CALL(*client_mock, DoPut) + ON_CALL(sql_client, DoPut) .WillByDefault([&buffer](const FlightCallOptions& options, const FlightDescriptor& descriptor1, const std::shared_ptr& schema, @@ -488,13 +450,13 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( return Status::OK(); }); if (schema == NULLPTR) { - EXPECT_CALL(*client_mock, DoPut(_, _, _, _, _)); + EXPECT_CALL(sql_client, DoPut(_, _, _, _, _)); } else { - EXPECT_CALL(*client_mock, DoPut(_, _, *schema, _, _)); + EXPECT_CALL(sql_client, DoPut(_, _, *schema, _, _)); } - ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client->Prepare(call_options, query)); - func(prepared_statement, *client_mock, schema, expected_rows); + 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); } @@ -502,7 +464,7 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateNoParameterBinding) { AssertTestPreparedStatementExecuteUpdateOk( [](const std::shared_ptr& prepared_statement, - internal::FlightClientImpl& client_mock, const std::shared_ptr* schema, + FlightSqlClient& sql_client, const std::shared_ptr* schema, const int64_t& row_count) {}, NULLPTR, sql_client); } @@ -512,7 +474,7 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBindi {arrow::field("field0", arrow::utf8()), arrow::field("field1", arrow::uint8())}); AssertTestPreparedStatementExecuteUpdateOk( [](const std::shared_ptr& prepared_statement, - internal::FlightClientImpl& client_mock, const std::shared_ptr* schema, + FlightSqlClient& sql_client, const std::shared_ptr* schema, const int64_t& row_count) { std::shared_ptr string_array; std::shared_ptr uint8_array; From 5f5ea06da8b7c62ea9419add7750f37fdedbf87f Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Wed, 10 Nov 2021 08:37:05 -0300 Subject: [PATCH 197/237] Fix wrong parameter name on SetParameters docs on client.h --- cpp/src/arrow/flight/flight_sql/client.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight_sql/client.h b/cpp/src/arrow/flight/flight_sql/client.h index 4b035778495..9afb8e2c743 100644 --- a/cpp/src/arrow/flight/flight_sql/client.h +++ b/cpp/src/arrow/flight/flight_sql/client.h @@ -232,7 +232,7 @@ class PreparedStatement { 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. + /// \param parameter_binding The parameters that will be bind. /// \return Status. Status SetParameters(std::shared_ptr parameter_binding); From 5e1c200bffcec0e7f3a4038177233f5df9e46957 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Wed, 10 Nov 2021 08:38:32 -0300 Subject: [PATCH 198/237] Fix linter erros on FindSQLite3Alt.cmake --- cpp/cmake_modules/FindSQLite3Alt.cmake | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/cpp/cmake_modules/FindSQLite3Alt.cmake b/cpp/cmake_modules/FindSQLite3Alt.cmake index 559931b1b7d..6c657eadf7b 100644 --- a/cpp/cmake_modules/FindSQLite3Alt.cmake +++ b/cpp/cmake_modules/FindSQLite3Alt.cmake @@ -32,11 +32,12 @@ 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 FAULT_MSG SQLite3_LIBRARY SQLite3_INCLUDE_DIR) +find_package_handle_standard_args(SQLite3Alt FAULT_MSG SQLite3_LIBRARY + SQLite3_INCLUDE_DIR) mark_as_advanced(SQLite3_LIBRARY SQLite3_INCLUDE_DIR) if(SQLite3_FOUND) - SET(SQLite3_INCLUDE_DIRS ${SQLite3_INCLUDE_DIR}) - SET(SQLite3_LIBRARIES ${SQLite3_LIBRARY}) + set(SQLite3_INCLUDE_DIRS ${SQLite3_INCLUDE_DIR}) + set(SQLite3_LIBRARIES ${SQLite3_LIBRARY}) endif() From 35a530ae5a8e0ca2dd5374bbddc94ebe10aef745 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Wed, 10 Nov 2021 12:13:15 -0300 Subject: [PATCH 199/237] Fix FindSQLite3Alt.cmake --- cpp/cmake_modules/FindSQLite3Alt.cmake | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/cpp/cmake_modules/FindSQLite3Alt.cmake b/cpp/cmake_modules/FindSQLite3Alt.cmake index 6c657eadf7b..4e4704a2db1 100644 --- a/cpp/cmake_modules/FindSQLite3Alt.cmake +++ b/cpp/cmake_modules/FindSQLite3Alt.cmake @@ -37,7 +37,5 @@ find_package_handle_standard_args(SQLite3Alt FAULT_MSG SQLite3_LIBRARY mark_as_advanced(SQLite3_LIBRARY SQLite3_INCLUDE_DIR) -if(SQLite3_FOUND) - set(SQLite3_INCLUDE_DIRS ${SQLite3_INCLUDE_DIR}) - set(SQLite3_LIBRARIES ${SQLite3_LIBRARY}) -endif() +set(SQLite3_INCLUDE_DIRS ${SQLite3_INCLUDE_DIR}) +set(SQLite3_LIBRARIES ${SQLite3_LIBRARY}) From df1ce715fe6382979ceaa11cfc6e66d5219a5562 Mon Sep 17 00:00:00 2001 From: Jose Almeida <53087160+jcralmeida@users.noreply.github.com> Date: Wed, 10 Nov 2021 13:32:53 -0300 Subject: [PATCH 200/237] [CPP] Modify the way arrays are created in tests (#199) * Create array using method ArrayFromJson * Create array using method ArrayFromJson in server tests * improve conditional from if statement --- .../arrow/flight/flight_sql/client_test.cc | 17 +++++---------- .../arrow/flight/flight_sql/server_test.cc | 21 +++++++------------ .../arrow/flight/flight_sql/test_app_cli.cc | 2 +- 3 files changed, 14 insertions(+), 26 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index 2b7c6025125..88928d0a56e 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -336,12 +336,7 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { auto parameter_schema = prepared_statement->parameter_schema(); - arrow::Int64Builder int_builder; - ASSERT_OK(int_builder.Append(1)); - std::shared_ptr int_array; - ASSERT_OK(int_builder.Finish(&int_array)); - std::shared_ptr result; - result = arrow::RecordBatch::Make(parameter_schema, 1, {int_array}); + auto result = RecordBatchFromJSON(parameter_schema, "[[1]]"); ASSERT_OK(prepared_statement->SetParameters(result)); EXPECT_CALL(sql_client, GetFlightInfo(_, _, _)); @@ -459,6 +454,7 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( 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) { @@ -476,12 +472,9 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBindi [](const std::shared_ptr& prepared_statement, FlightSqlClient& sql_client, const std::shared_ptr* schema, const int64_t& row_count) { - std::shared_ptr string_array; - std::shared_ptr uint8_array; - const std::vector string_data{"Lorem", "Ipsum", "Foo", "Bar", "Baz"}; - const std::vector uint8_data{0, 10, 15, 20, 25}; - ArrayFromVector(string_data, &string_array); - ArrayFromVector(uint8_data, &uint8_array); + + 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)); diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 9e9fd9f72b2..ab172577779 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -85,15 +85,10 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { arrow::schema({arrow::field("id", int64()), arrow::field("keyName", utf8()), arrow::field("value", int64()), arrow::field("foreignId", int64())}); - std::shared_ptr id_array; - std::shared_ptr keyname_array; - std::shared_ptr value_array; - std::shared_ptr foreignId_array; - ArrayFromVector({1, 2, 3}, &id_array); - ArrayFromVector({"one", "zero", "negative one"}, - &keyname_array); - ArrayFromVector({1, 0, -1}, &value_array); - ArrayFromVector({1, 1, 1}, &foreignId_array); + std::shared_ptr id_array = ArrayFromJSON(int64(), R"([1, 2, 3])"); + std::shared_ptr keyname_array = ArrayFromJSON(utf8(), R"(["one", "zero", "negative one"])"); + std::shared_ptr value_array = ArrayFromJSON(int64(), R"([1, 0, -1])"); + std::shared_ptr foreignId_array = ArrayFromJSON(int64(), R"([1, 1, 1])"); const std::shared_ptr
& expected_table = Table::Make( expected_schema, {id_array, keyname_array, value_array, foreignId_array}); @@ -226,10 +221,10 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { ArrayFromVector({"table"}, &table_type); const std::shared_ptr schema_table = - arrow::schema({arrow::field("id", int64(), true, NULL), - arrow::field("keyName", utf8(), true, NULL), - arrow::field("value", int64(), true, NULL), - arrow::field("foreignId", int64(), true, NULL)}); + arrow::schema({arrow::field("id", int64(), true), + arrow::field("keyName", utf8(), true), + arrow::field("value", int64(), true), + arrow::field("foreignId", int64(), true)}); const arrow::Result>& value = ipc::SerializeSchema(*schema_table); diff --git a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc index fb2a8890bfb..b98ff07d67a 100644 --- a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc @@ -73,7 +73,7 @@ Status PrintResultsForEndpoint(FlightSqlClient& client, while (true) { ARROW_RETURN_NOT_OK(stream->Next(&chunk)); - if (!(chunk.data != nullptr)) { + if (chunk.data) { break; } std::cout << chunk.data->ToString() << std::endl; From 0519976bc9299607444c0f21141b939da6142da0 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 10 Nov 2021 14:23:05 -0300 Subject: [PATCH 201/237] Update vcpkg.json and remove unused variable on CMakeLists.txt (#200) --- cpp/src/arrow/flight/flight_sql/CMakeLists.txt | 5 ----- cpp/src/arrow/flight/flight_sql/client_test.cc | 1 - cpp/vcpkg.json | 1 + 3 files changed, 1 insertion(+), 6 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index fd28559a91b..91e4174594b 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -59,15 +59,10 @@ add_arrow_lib(arrow_flight_sql arrow_flight_static) if(ARROW_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}) 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_TEST_LINK_LIBS arrow_flight_shared arrow_flight_testing_shared - ${ARROW_TEST_LINK_LIBS}) set(ARROW_FLIGHT_SQL_TEST_LINK_LIBS arrow_flight_sql_shared arrow_flight_testing_shared ${ARROW_TEST_LINK_LIBS}) endif() diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index 88928d0a56e..970c10ce5f7 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -472,7 +472,6 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBindi [](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 = diff --git a/cpp/vcpkg.json b/cpp/vcpkg.json index d9505b67383..11ff74bf841 100644 --- a/cpp/vcpkg.json +++ b/cpp/vcpkg.json @@ -19,6 +19,7 @@ "boost-multiprecision", "boost-process", "boost-system", + "boost-uuid", "brotli", "bzip2", "c-ares", From 93f93c4c1148b394b660461f20a8a7511ff23546 Mon Sep 17 00:00:00 2001 From: Jose Almeida <53087160+jcralmeida@users.noreply.github.com> Date: Wed, 10 Nov 2021 14:27:18 -0300 Subject: [PATCH 202/237] [CPP] Fix issues from client files in flight-sql (#201) * Order include and rename protocol namespace * Change a reference to pointer in the client client and refactor the name of namespace on functions * Nit: remove empty line * Improve naming from protocol::sql namespace * Fix include and its orderding --- cpp/src/arrow/flight/flight_sql/api.h | 2 +- cpp/src/arrow/flight/flight_sql/client.cc | 79 ++++++++++--------- cpp/src/arrow/flight/flight_sql/client.h | 23 +++--- .../arrow/flight/flight_sql/client_test.cc | 9 ++- .../flight_sql/example/sqlite_server.cc | 14 ++-- .../flight/flight_sql/example/sqlite_server.h | 9 ++- .../flight_sql/example/sqlite_statement.cc | 6 +- .../flight_sql/example/sqlite_statement.h | 3 +- .../example/sqlite_statement_batch_reader.cc | 8 +- .../example/sqlite_statement_batch_reader.h | 5 +- .../sqlite_tables_schema_batch_reader.cc | 14 ++-- .../sqlite_tables_schema_batch_reader.h | 7 +- cpp/src/arrow/flight/flight_sql/server.cc | 10 ++- cpp/src/arrow/flight/flight_sql/server.h | 12 +-- .../arrow/flight/flight_sql/server_test.cc | 31 ++++---- .../arrow/flight/flight_sql/test_app_cli.cc | 16 ++-- .../flight/flight_sql/test_server_cli.cc | 14 ++-- 17 files changed, 140 insertions(+), 122 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/api.h b/cpp/src/arrow/flight/flight_sql/api.h index ddb83b25b1c..8d5124ee86d 100644 --- a/cpp/src/arrow/flight/flight_sql/api.h +++ b/cpp/src/arrow/flight/flight_sql/api.h @@ -17,4 +17,4 @@ #pragma once -#include +#include "arrow/flight/flight_sql/client.h" diff --git a/cpp/src/arrow/flight/flight_sql/client.cc b/cpp/src/arrow/flight/flight_sql/client.cc index 77891e55536..8d73a6080da 100644 --- a/cpp/src/arrow/flight/flight_sql/client.cc +++ b/cpp/src/arrow/flight/flight_sql/client.cc @@ -15,18 +15,19 @@ // specific language governing permissions and limitations // under the License. -#include -#include -#include -#include -#include -#include -#include -#include -#include #include -namespace pb = arrow::flight::protocol; +#include "arrow/buffer.h" +#include "arrow/flight/flight_sql/client.h" +#include "arrow/flight/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 { @@ -35,7 +36,7 @@ namespace sql { FlightSqlClient::FlightSqlClient(std::shared_ptr client) : impl_(std::move(client)) {} -PreparedStatement::PreparedStatement(FlightSqlClient& client, std::string handle, +PreparedStatement::PreparedStatement(FlightSqlClient *client, std::string handle, std::shared_ptr dataset_schema, std::shared_ptr parameter_schema, FlightCallOptions options) @@ -79,7 +80,7 @@ arrow::Result> GetFlightInfoForCommand( arrow::Result> FlightSqlClient::Execute( const FlightCallOptions& options, const std::string& query) { - pb::sql::CommandStatementQuery command; + flight_sql_pb::CommandStatementQuery command; command.set_query(query); return GetFlightInfoForCommand(*this, options, command); @@ -87,7 +88,7 @@ arrow::Result> FlightSqlClient::Execute( arrow::Result FlightSqlClient::ExecuteUpdate(const FlightCallOptions& options, const std::string& query) { - pb::sql::CommandStatementUpdate command; + flight_sql_pb::CommandStatementUpdate command; command.set_query(query); const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); @@ -101,9 +102,9 @@ arrow::Result FlightSqlClient::ExecuteUpdate(const FlightCallOptions& o ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata)); - pb::sql::DoPutUpdateResult doPutUpdateResult; + flight_sql_pb::DoPutUpdateResult doPutUpdateResult; - pb::sql::DoPutUpdateResult result; + flight_sql_pb::DoPutUpdateResult result; if (!result.ParseFromArray(metadata->data(), static_cast(metadata->size()))) { return Status::Invalid("Unable to parse DoPutUpdateResult object."); } @@ -113,7 +114,7 @@ arrow::Result FlightSqlClient::ExecuteUpdate(const FlightCallOptions& o arrow::Result> FlightSqlClient::GetCatalogs( const FlightCallOptions& options) { - pb::sql::CommandGetCatalogs command; + flight_sql_pb::CommandGetCatalogs command; return GetFlightInfoForCommand(*this, options, command); } @@ -121,7 +122,7 @@ arrow::Result> FlightSqlClient::GetCatalogs( arrow::Result> FlightSqlClient::GetSchemas( const FlightCallOptions& options, const std::string* catalog, const std::string* schema_filter_pattern) { - pb::sql::CommandGetSchemas command; + flight_sql_pb::CommandGetSchemas command; if (catalog != NULLPTR) { command.set_catalog(*catalog); } @@ -133,10 +134,10 @@ arrow::Result> FlightSqlClient::GetSchemas( } arrow::Result> FlightSqlClient::GetTables( - const FlightCallOptions& options, const std::string* catalog, - const std::string* schema_filter_pattern, const std::string* table_filter_pattern, - bool include_schema, std::vector& table_types) { - pb::sql::CommandGetTables command; + const FlightCallOptions& options, const std::string* catalog, + const std::string* 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); @@ -162,7 +163,7 @@ arrow::Result> FlightSqlClient::GetTables( arrow::Result> FlightSqlClient::GetPrimaryKeys( const FlightCallOptions& options, const std::string* catalog, const std::string* schema, const std::string& table) { - pb::sql::CommandGetPrimaryKeys command; + flight_sql_pb::CommandGetPrimaryKeys command; if (catalog != NULLPTR) { command.set_catalog(*catalog); @@ -180,7 +181,7 @@ arrow::Result> FlightSqlClient::GetPrimaryKeys( arrow::Result> FlightSqlClient::GetExportedKeys( const FlightCallOptions& options, const std::string* catalog, const std::string* schema, const std::string& table) { - pb::sql::CommandGetExportedKeys command; + flight_sql_pb::CommandGetExportedKeys command; if (catalog != NULLPTR) { command.set_catalog(*catalog); @@ -198,7 +199,7 @@ arrow::Result> FlightSqlClient::GetExportedKeys( arrow::Result> FlightSqlClient::GetImportedKeys( const FlightCallOptions& options, const std::string* catalog, const std::string* schema, const std::string& table) { - pb::sql::CommandGetImportedKeys command; + flight_sql_pb::CommandGetImportedKeys command; if (catalog != NULLPTR) { command.set_catalog(*catalog); @@ -218,7 +219,7 @@ arrow::Result> FlightSqlClient::GetCrossReference( const std::string* pk_schema, const std::string& pk_table, const std::string* fk_catalog, const std::string* fk_schema, const std::string& fk_table) { - pb::sql::CommandGetCrossReference command; + flight_sql_pb::CommandGetCrossReference command; if (pk_catalog != NULLPTR) { command.set_pk_catalog(*pk_catalog); @@ -241,7 +242,7 @@ arrow::Result> FlightSqlClient::GetCrossReference( arrow::Result> FlightSqlClient::GetTableTypes( const FlightCallOptions& options) { - pb::sql::CommandGetTableTypes command; + flight_sql_pb::CommandGetTableTypes command; return GetFlightInfoForCommand(*this, options, command); } @@ -257,7 +258,7 @@ arrow::Result> FlightSqlClient::DoGet( arrow::Result> FlightSqlClient::Prepare( const FlightCallOptions& options, const std::string& query) { google::protobuf::Any command; - pb::sql::ActionCreatePreparedStatementRequest request; + flight_sql_pb::ActionCreatePreparedStatementRequest request; request.set_query(query); command.PackFrom(request); @@ -280,7 +281,7 @@ arrow::Result> FlightSqlClient::Prepare( return Status::Invalid("Unable to parse packed ActionCreatePreparedStatementResult"); } - pb::sql::ActionCreatePreparedStatementResult prepared_statement_result; + flight_sql_pb::ActionCreatePreparedStatementResult prepared_statement_result; if (!prepared_result.UnpackTo(&prepared_statement_result)) { return Status::Invalid("Unable to unpack ActionCreatePreparedStatementResult"); @@ -306,7 +307,7 @@ arrow::Result> FlightSqlClient::Prepare( } auto handle = prepared_statement_result.prepared_statement_handle(); - return std::make_shared(*this, handle, dataset_schema, + return std::make_shared(this, handle, dataset_schema, parameter_schema, options); } @@ -315,7 +316,7 @@ arrow::Result> PreparedStatement::Execute() { return Status::Invalid("Statement already closed."); } - pb::sql::CommandPreparedStatementQuery execute_query_command; + flight_sql_pb::CommandPreparedStatementQuery execute_query_command; execute_query_command.set_prepared_statement_handle(handle_); @@ -328,7 +329,7 @@ arrow::Result> PreparedStatement::Execute() { 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(), + ARROW_RETURN_NOT_OK(client_->DoPut(options_, descriptor, parameter_binding_->schema(), &writer, &reader)); ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*parameter_binding_)); @@ -339,7 +340,7 @@ arrow::Result> PreparedStatement::Execute() { } std::unique_ptr info; - ARROW_RETURN_NOT_OK(client_.GetFlightInfo(options_, descriptor, &info)); + ARROW_RETURN_NOT_OK(client_->GetFlightInfo(options_, descriptor, &info)); return std::move(info); } @@ -349,19 +350,19 @@ arrow::Result PreparedStatement::ExecuteUpdate() { return Status::Invalid("Statement already closed."); } - pb::sql::CommandPreparedStatementUpdate command; + 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(), + 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)); + 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)); @@ -372,7 +373,7 @@ arrow::Result PreparedStatement::ExecuteUpdate() { ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata)); ARROW_RETURN_NOT_OK(writer->Close()); - pb::sql::DoPutUpdateResult result; + flight_sql_pb::DoPutUpdateResult result; if (!result.ParseFromArray(metadata->data(), static_cast(metadata->size()))) { return Status::Invalid("Unable to parse DoPutUpdateResult object."); } @@ -401,7 +402,7 @@ Status PreparedStatement::Close() { return Status::Invalid("Statement already closed."); } google::protobuf::Any command; - pb::sql::ActionClosePreparedStatementRequest request; + flight_sql_pb::ActionClosePreparedStatementRequest request; request.set_prepared_statement_handle(handle_); command.PackFrom(request); @@ -412,7 +413,7 @@ Status PreparedStatement::Close() { std::unique_ptr results; - ARROW_RETURN_NOT_OK(client_.DoAction(options_, action, &results)); + ARROW_RETURN_NOT_OK(client_->DoAction(options_, action, &results)); is_closed_ = true; @@ -421,7 +422,7 @@ Status PreparedStatement::Close() { arrow::Result> FlightSqlClient::GetSqlInfo( const FlightCallOptions& options, const std::vector& sql_info) { - pb::sql::CommandGetSqlInfo command; + flight_sql_pb::CommandGetSqlInfo command; for (const int& info : sql_info) command.add_info(info); return GetFlightInfoForCommand(*this, options, command); diff --git a/cpp/src/arrow/flight/flight_sql/client.h b/cpp/src/arrow/flight/flight_sql/client.h index 9afb8e2c743..91197b98578 100644 --- a/cpp/src/arrow/flight/flight_sql/client.h +++ b/cpp/src/arrow/flight/flight_sql/client.h @@ -17,14 +17,14 @@ #pragma once -#include -#include -#include -#include - #include #include +#include "arrow/flight/client.h" +#include "arrow/flight/types.h" +#include "arrow/result.h" +#include "arrow/status.h" + namespace arrow { namespace flight { namespace sql { @@ -90,9 +90,9 @@ class ARROW_EXPORT FlightSqlClient { /// \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* schema_filter_pattern, const std::string* table_filter_pattern, - bool include_schema, std::vector& table_types); + const FlightCallOptions& options, const std::string* catalog, + const std::string* 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. @@ -188,8 +188,9 @@ class ARROW_EXPORT FlightSqlClient { } }; -class PreparedStatement { - FlightSqlClient& client_; +/// \brief PreparedStatement class from flight sql. +class ARROW_EXPORT PreparedStatement { + FlightSqlClient* client_; FlightCallOptions options_; std::string handle_; std::shared_ptr dataset_schema_; @@ -204,7 +205,7 @@ class PreparedStatement { /// \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, + PreparedStatement(FlightSqlClient *client, std::string handle, std::shared_ptr dataset_schema, std::shared_ptr parameter_schema, FlightCallOptions options); diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index 970c10ce5f7..43dd90d19f2 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -15,16 +15,17 @@ // specific language governing permissions and limitations // under the License. -#include -#include -#include -#include #include #include #include #include +#include "arrow/flight/client.h" +#include "arrow/flight/flight_sql/FlightSql.pb.h" +#include "arrow/flight/flight_sql/api.h" +#include "arrow/testing/gtest_util.h" + namespace pb = arrow::flight::protocol; using ::testing::_; using ::testing::Ref; diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc index 86f727762dd..d9afdd052a4 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc @@ -15,12 +15,8 @@ // specific language governing permissions and limitations // under the License. -#include -#include -#include -#include -#include -#include +#include "arrow/flight/flight_sql/example/sqlite_server.h" + #include #include @@ -29,6 +25,12 @@ #include #include +#include "arrow/api.h" +#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h" +#include "arrow/flight/flight_sql/server.h" + namespace arrow { namespace flight { namespace sql { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h index 99d1132c85d..69f0afb0c6e 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h @@ -17,10 +17,6 @@ #pragma once -#include -#include -#include -#include #include #include @@ -29,6 +25,11 @@ #include #include +#include "arrow/api.h" +#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/flight_sql/server.h" + namespace arrow { namespace flight { namespace sql { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc index 496841f8d5c..f0dfc8da724 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc @@ -15,12 +15,14 @@ // specific language governing permissions and limitations // under the License. -#include -#include +#include "arrow/flight/flight_sql/example/sqlite_statement.h" + #include #include +#include "arrow/flight/flight_sql/example/sqlite_server.h" + namespace arrow { namespace flight { namespace sql { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h index 39430fb43df..0d32a2f282d 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h @@ -17,12 +17,13 @@ #pragma once -#include #include #include #include +#include "arrow/type_fwd.h" + namespace arrow { namespace flight { namespace sql { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc index 9b9e193cc33..149799a7ea0 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc @@ -15,11 +15,13 @@ // specific language governing permissions and limitations // under the License. -#include -#include -#include +#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" + #include +#include "arrow/api.h" +#include "arrow/flight/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); \ diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h index 8bd8c72cb09..84064a8daaa 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h @@ -17,12 +17,13 @@ #pragma once -#include -#include #include #include +#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/record_batch.h" + namespace arrow { namespace flight { namespace sql { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc index 42c28ae1a10..d9f16dce94c 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc @@ -15,16 +15,18 @@ // specific language governing permissions and limitations // under the License. -#include -#include -#include -#include -#include -#include +#include "arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h" + #include #include +#include "arrow/flight/flight_sql/example/sqlite_server.h" +#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/server.h" +#include "arrow/ipc/writer.h" +#include "arrow/record_batch.h" + namespace arrow { namespace flight { namespace sql { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h index 99c7486d0e3..e4fcb6680c7 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h @@ -17,14 +17,15 @@ #pragma once -#include -#include -#include #include #include #include +#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" +#include "arrow/record_batch.h" + namespace arrow { namespace flight { namespace sql { diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index 91c09ae878b..c485006c7b9 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -18,16 +18,18 @@ // Interfaces to use for defining Flight RPC servers. API should be considered // experimental for now -#include -#include -#include -#include +#include "arrow/flight/flight_sql/server.h" + #include #include #include #include +#include "arrow/api.h" +#include "arrow/buffer.h" +#include "arrow/flight/flight_sql/FlightSql.pb.h" + #define PROPERTY_TO_OPTIONAL(COMMAND, PROPERTY) \ COMMAND.has_##PROPERTY() ? util::make_optional(COMMAND.PROPERTY()) : util::nullopt; diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index 2e667311fc4..031e1135152 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -20,15 +20,15 @@ #pragma once -#include -#include -#include -#include -#include - #include #include +#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/flight_sql/server.h" +#include "arrow/flight/server.h" +#include "arrow/util/optional.h" + namespace arrow { namespace flight { namespace sql { diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index ab172577779..ff22e9088c1 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -15,18 +15,20 @@ // specific language governing permissions and limitations // under the License. -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include "arrow/flight/flight_sql/server.h" + #include #include +#include "arrow/api.h" +#include "arrow/flight/api.h" +#include "arrow/flight/flight_sql/FlightSql.pb.h" +#include "arrow/flight/flight_sql/api.h" +#include "arrow/flight/flight_sql/example/sqlite_server.h" +#include "arrow/flight/test_util.h" +#include "arrow/flight/types.h" +#include "arrow/testing/gtest_util.h" + using ::testing::_; using ::testing::Ref; @@ -86,7 +88,8 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { arrow::field("value", int64()), arrow::field("foreignId", int64())}); std::shared_ptr id_array = ArrayFromJSON(int64(), R"([1, 2, 3])"); - std::shared_ptr keyname_array = ArrayFromJSON(utf8(), R"(["one", "zero", "negative one"])"); + std::shared_ptr keyname_array = + ArrayFromJSON(utf8(), R"(["one", "zero", "negative one"])"); std::shared_ptr value_array = ArrayFromJSON(int64(), R"([1, 0, -1])"); std::shared_ptr foreignId_array = ArrayFromJSON(int64(), R"([1, 1, 1])"); @@ -220,11 +223,9 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { ArrayFromVector({"intTable"}, &table_name); ArrayFromVector({"table"}, &table_type); - 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)}); + 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)}); const arrow::Result>& value = ipc::SerializeSchema(*schema_table); diff --git a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc index b98ff07d67a..b73181c16f0 100644 --- a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc @@ -15,14 +15,14 @@ // specific language governing permissions and limitations // under the License. -#include -#include -#include -#include -#include -#include -#include -#include +#include "arrow/array/builder_binary.h" +#include "arrow/array/builder_primitive.h" +#include "arrow/flight/api.h" +#include "arrow/flight/flight_sql/api.h" +#include "arrow/io/memory.h" +#include "arrow/pretty_print.h" +#include "arrow/status.h" +#include "arrow/table.h" #include #include diff --git a/cpp/src/arrow/flight/flight_sql/test_server_cli.cc b/cpp/src/arrow/flight/flight_sql/test_server_cli.cc index 220a975a6d8..4a356c56fa5 100644 --- a/cpp/src/arrow/flight/flight_sql/test_server_cli.cc +++ b/cpp/src/arrow/flight/flight_sql/test_server_cli.cc @@ -15,13 +15,13 @@ // specific language governing permissions and limitations // under the License. -#include -#include -#include -#include -#include -#include -#include +#include "arrow/flight/flight_sql/example/sqlite_server.h" +#include "arrow/flight/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" #include #include From 817baf9a5d347fc5d8cbf9c598458fbb85acfc3a Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 10 Nov 2021 15:05:20 -0300 Subject: [PATCH 203/237] Fix code style issues --- cpp/src/arrow/flight/flight_sql/client.cc | 15 ++++++++------- cpp/src/arrow/flight/flight_sql/client.h | 8 ++++---- cpp/src/arrow/flight/flight_sql/client_test.cc | 6 ++++-- cpp/src/arrow/flight/flight_sql/test_app_cli.cc | 11 ++++++----- .../arrow/flight/flight_sql/test_server_cli.cc | 11 ++++++----- 5 files changed, 28 insertions(+), 23 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/client.cc b/cpp/src/arrow/flight/flight_sql/client.cc index 8d73a6080da..82fc73104b0 100644 --- a/cpp/src/arrow/flight/flight_sql/client.cc +++ b/cpp/src/arrow/flight/flight_sql/client.cc @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. +#include "arrow/flight/flight_sql/client.h" + #include #include "arrow/buffer.h" -#include "arrow/flight/flight_sql/client.h" #include "arrow/flight/flight_sql/FlightSql.pb.h" #include "arrow/flight/types.h" #include "arrow/io/memory.h" @@ -36,7 +37,7 @@ namespace sql { FlightSqlClient::FlightSqlClient(std::shared_ptr client) : impl_(std::move(client)) {} -PreparedStatement::PreparedStatement(FlightSqlClient *client, std::string handle, +PreparedStatement::PreparedStatement(FlightSqlClient* client, std::string handle, std::shared_ptr dataset_schema, std::shared_ptr parameter_schema, FlightCallOptions options) @@ -134,9 +135,9 @@ arrow::Result> FlightSqlClient::GetSchemas( } arrow::Result> FlightSqlClient::GetTables( - const FlightCallOptions& options, const std::string* catalog, - const std::string* schema_filter_pattern, const std::string* table_filter_pattern, - bool include_schema, const std::vector &table_types) { + const FlightCallOptions& options, const std::string* catalog, + const std::string* 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) { @@ -330,7 +331,7 @@ arrow::Result> PreparedStatement::Execute() { std::unique_ptr writer; std::unique_ptr reader; ARROW_RETURN_NOT_OK(client_->DoPut(options_, descriptor, parameter_binding_->schema(), - &writer, &reader)); + &writer, &reader)); ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*parameter_binding_)); ARROW_RETURN_NOT_OK(writer->DoneWriting()); @@ -358,7 +359,7 @@ arrow::Result PreparedStatement::ExecuteUpdate() { if (parameter_binding_ && parameter_binding_->num_rows() > 0) { ARROW_RETURN_NOT_OK(client_->DoPut(options_, descriptor, parameter_binding_->schema(), - &writer, &reader)); + &writer, &reader)); ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*parameter_binding_)); } else { const std::shared_ptr schema = arrow::schema({}); diff --git a/cpp/src/arrow/flight/flight_sql/client.h b/cpp/src/arrow/flight/flight_sql/client.h index 91197b98578..c3c571d6b8b 100644 --- a/cpp/src/arrow/flight/flight_sql/client.h +++ b/cpp/src/arrow/flight/flight_sql/client.h @@ -90,9 +90,9 @@ class ARROW_EXPORT FlightSqlClient { /// \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* schema_filter_pattern, const std::string* table_filter_pattern, - bool include_schema, const std::vector &table_types); + const FlightCallOptions& options, const std::string* catalog, + const std::string* 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. @@ -205,7 +205,7 @@ class ARROW_EXPORT PreparedStatement { /// \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, + PreparedStatement(FlightSqlClient* client, std::string handle, std::shared_ptr dataset_schema, std::shared_ptr parameter_schema, FlightCallOptions options); diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/flight_sql/client_test.cc index 43dd90d19f2..0397ff91489 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/flight_sql/client_test.cc @@ -15,13 +15,14 @@ // specific language governing permissions and limitations // under the License. +#include "arrow/flight/client.h" + #include #include #include #include -#include "arrow/flight/client.h" #include "arrow/flight/flight_sql/FlightSql.pb.h" #include "arrow/flight/flight_sql/api.h" #include "arrow/testing/gtest_util.h" @@ -473,7 +474,8 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBindi [](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 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}); diff --git a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc index b73181c16f0..91b57f99319 100644 --- a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc @@ -15,6 +15,12 @@ // 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" @@ -23,11 +29,6 @@ #include "arrow/pretty_print.h" #include "arrow/status.h" #include "arrow/table.h" -#include - -#include -#include -#include using arrow::Result; using arrow::Schema; diff --git a/cpp/src/arrow/flight/flight_sql/test_server_cli.cc b/cpp/src/arrow/flight/flight_sql/test_server_cli.cc index 4a356c56fa5..22c7610c8dd 100644 --- a/cpp/src/arrow/flight/flight_sql/test_server_cli.cc +++ b/cpp/src/arrow/flight/flight_sql/test_server_cli.cc @@ -15,6 +15,12 @@ // specific language governing permissions and limitations // under the License. +#include + +#include +#include +#include + #include "arrow/flight/flight_sql/example/sqlite_server.h" #include "arrow/flight/server.h" #include "arrow/flight/test_integration.h" @@ -22,11 +28,6 @@ #include "arrow/io/test_common.h" #include "arrow/testing/json_integration.h" #include "arrow/util/logging.h" -#include - -#include -#include -#include DEFINE_int32(port, 31337, "Server port to listen on"); From 404b27deb0df2833175a58efe63eaf718e50ffd1 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Wed, 10 Nov 2021 17:29:01 -0300 Subject: [PATCH 204/237] [C++] Implement GetSqlInfo on server example (#193) * WIP: Implement GetSqlInfo on server-side * Fix build and missing code parts * Add test for map> @ GetSqlInfo * Fix integration tests * Fix comments pointed on review * Fix comments pointed on review * Add more comments about the logic around DenseUnionArrays * Remove unnecessary includes on sqlite_server.cc * Fix comments pointed on review * Use std::vector reserve and assign to avoid allocating temporary object * Remove unused dependencies on server.cc Co-authored-by: Abner Eduardo Ferreira --- .../arrow/flight/flight_sql/CMakeLists.txt | 3 +- .../flight_sql/example/sqlite_server.cc | 81 ++++---- .../flight/flight_sql/example/sqlite_server.h | 192 +++++++++++++++++- cpp/src/arrow/flight/flight_sql/server.cc | 121 ++++++++++- cpp/src/arrow/flight/flight_sql/server.h | 23 ++- .../arrow/flight/flight_sql/server_test.cc | 115 +++++++++++ .../arrow/flight/flight_sql/sql_info_util.cc | 113 +++++++++++ .../arrow/flight/flight_sql/sql_info_util.h | 84 ++++++++ .../arrow/flight/flight_sql/test_app_cli.cc | 2 + format/FlightSql.proto | 43 +++- 10 files changed, 718 insertions(+), 59 deletions(-) create mode 100644 cpp/src/arrow/flight/flight_sql/sql_info_util.cc create mode 100644 cpp/src/arrow/flight/flight_sql/sql_info_util.h diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index 91e4174594b..61d2814a9b6 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -36,7 +36,7 @@ set_source_files_properties(${FLIGHT_SQL_GENERATED_PROTO_FILES} PROPERTIES GENER add_custom_target(flight_sql_protobuf_gen ALL DEPENDS ${FLIGHT_SQL_GENERATED_PROTO_FILES}) -set(ARROW_FLIGHT_SQL_SRCS server.cc client.cc) +set(ARROW_FLIGHT_SQL_SRCS server.cc sql_info_util.cc client.cc) add_arrow_lib(arrow_flight_sql CMAKE_PACKAGE_NAME @@ -88,6 +88,7 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) add_executable(flight_sql_test_server server.cc + sql_info_util.cc test_server_cli.cc example/sqlite_statement.cc example/sqlite_statement_batch_reader.cc diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc index d9afdd052a4..4b7e40aee4f 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc @@ -21,9 +21,7 @@ #include #include -#include #include -#include #include "arrow/api.h" #include "arrow/flight/flight_sql/example/sqlite_statement.h" @@ -36,25 +34,21 @@ namespace flight { namespace sql { namespace example { -std::shared_ptr GetArrowType(const char* sqlite_type) { - if (sqlite_type == NULLPTR) { - // SQLite may not know the column type yet. - return null(); - } +namespace { - 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 { - return null(); +/// \brief Gets a SqliteStatement by given handle +arrow::Result> GetStatementByHandle( + const std::map>& + prepared_statements, + const std::string& handle) { + const auto& uuid = boost::lexical_cast(handle); + + auto search = prepared_statements.find(uuid); + if (search == prepared_statements.end()) { + return Status::Invalid("Prepared statement not found"); } + + return search->second; } std::string PrepareQueryForGetTables(const GetTables& command) { @@ -145,6 +139,29 @@ Status SetParametersOnSQLiteStatement(sqlite3_stmt* stmt, FlightMessageReader* r return Status::OK(); } +} // 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)); + } +} + SQLiteFlightSqlServer::SQLiteFlightSqlServer(sqlite3* db) : db_(db), uuid_generator_({}) { assert(db_); } @@ -165,6 +182,9 @@ arrow::Result> SQLiteFlightSqlServer::Cre } std::shared_ptr result(new SQLiteFlightSqlServer(db)); + for (const auto& id_to_result : GetSqlInfoIdToResult()) { + result->RegisterSqlInfo(id_to_result.first, id_to_result.second); + } ARROW_UNUSED(result->ExecuteSql(R"( CREATE TABLE foreignTable ( @@ -477,26 +497,12 @@ Status SQLiteFlightSqlServer::DoGetPreparedStatement( return Status::OK(); } -Status SQLiteFlightSqlServer::GetStatementByHandle( - const std::string& prepared_statement_handle, - std::shared_ptr* result) { - const auto& uuid = boost::lexical_cast(prepared_statement_handle); - - auto search = prepared_statements_.find(uuid); - if (search == prepared_statements_.end()) { - return Status::Invalid("Prepared statement not found"); - } - - *result = search->second; - return Status::OK(); -} - Status SQLiteFlightSqlServer::DoPutPreparedStatementQuery( const PreparedStatementQuery& command, const ServerCallContext& context, FlightMessageReader* reader, FlightMetadataWriter* writer) { const std::string& prepared_statement_handle = command.prepared_statement_handle; - std::shared_ptr statement; - ARROW_RETURN_NOT_OK(GetStatementByHandle(prepared_statement_handle, &statement)); + ARROW_ASSIGN_OR_RAISE(auto statement, GetStatementByHandle(prepared_statements_, + prepared_statement_handle)); sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); ARROW_RETURN_NOT_OK(SetParametersOnSQLiteStatement(stmt, reader)); @@ -508,9 +514,8 @@ arrow::Result SQLiteFlightSqlServer::DoPutPreparedStatementUpdate( const PreparedStatementUpdate& command, const ServerCallContext& context, FlightMessageReader* reader) { const std::string& prepared_statement_handle = command.prepared_statement_handle; - - std::shared_ptr statement; - ARROW_RETURN_NOT_OK(GetStatementByHandle(prepared_statement_handle, &statement)); + ARROW_ASSIGN_OR_RAISE(auto statement, GetStatementByHandle(prepared_statements_, + prepared_statement_handle)); sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); ARROW_RETURN_NOT_OK(SetParametersOnSQLiteStatement(stmt, reader)); diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h index 69f0afb0c6e..298268e3708 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h @@ -26,6 +26,7 @@ #include #include "arrow/api.h" +#include "arrow/flight/flight_sql/FlightSql.pb.h" #include "arrow/flight/flight_sql/example/sqlite_statement.h" #include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" #include "arrow/flight/flight_sql/server.h" @@ -35,6 +36,194 @@ namespace flight { namespace sql { namespace example { +namespace flight_sql_pb = arrow::flight::protocol::sql; + +/// \brief Gets the mapping from SQL info ids to SqlInfoResult instances. +/// \return the cache. +inline sql_info_id_to_result_t GetSqlInfoIdToResult() { + return {{flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_NAME, std::string("db_name")}, + {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_VERSION, std::string("sqlite 3")}, + {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_ARROW_VERSION, + std::string("7.0.0-SNAPSHOT" /* Only an example */)}, + {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_READ_ONLY, false}, + {flight_sql_pb::SqlInfo::SQL_DDL_CATALOG, + false /* SQLite 3 does not support catalogs */}, + {flight_sql_pb::SqlInfo::SQL_DDL_SCHEMA, + false /* SQLite 3 does not support schemas */}, + {flight_sql_pb::SqlInfo::SQL_DDL_TABLE, true}, + {flight_sql_pb::SqlInfo::SQL_IDENTIFIER_CASE, + int64_t(flight_sql_pb::SqlSupportedCaseSensitivity:: + SQL_CASE_SENSITIVITY_CASE_INSENSITIVE)}, + {flight_sql_pb::SqlInfo::SQL_IDENTIFIER_QUOTE_CHAR, std::string("\"")}, + {flight_sql_pb::SqlInfo::SQL_QUOTED_IDENTIFIER_CASE, + int64_t(flight_sql_pb::SqlSupportedCaseSensitivity:: + SQL_CASE_SENSITIVITY_CASE_INSENSITIVE)}, + {flight_sql_pb::SqlInfo::SQL_ALL_TABLES_ARE_SELECTABLE, true}, + {flight_sql_pb::SqlInfo::SQL_NULL_ORDERING, + int64_t(flight_sql_pb::SqlNullOrdering::SQL_NULLS_SORTED_AT_START)}, + {flight_sql_pb::SqlInfo::SQL_KEYWORDS, string_list_t({"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"})}, + {flight_sql_pb::SqlInfo::SQL_NUMERIC_FUNCTIONS, + string_list_t({"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"})}, + {flight_sql_pb::SqlInfo::SQL_STRING_FUNCTIONS, + string_list_t({"SUBSTR", "TRIM", "LTRIM", "RTRIM", "LENGTH", "REPLACE", + "UPPER", "LOWER", "INSTR"})}, + {flight_sql_pb::SqlInfo::SQL_SUPPORTS_CONVERT, + int32_to_int32_list_t( + {{flight_sql_pb::SqlSupportsConvert::SQL_CONVERT_BIGINT, + std::vector( + {flight_sql_pb::SqlSupportsConvert::SQL_CONVERT_INTEGER})}})}}; +} + /// \brief Convert a column type to a ArrowType. /// \param sqlite_type the sqlite type. /// \return The equivalent ArrowType. @@ -154,9 +343,6 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { /// SQLiteFlightSqlServer /// \param db The db parameter from SQLite. The Server it is taking the ownership. explicit SQLiteFlightSqlServer(sqlite3* db); - - Status GetStatementByHandle(const std::string& prepared_statement_handle, - std::shared_ptr* result); }; } // namespace example diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index c485006c7b9..6690c221e79 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -22,10 +22,6 @@ #include -#include -#include -#include - #include "arrow/api.h" #include "arrow/buffer.h" #include "arrow/flight/flight_sql/FlightSql.pb.h" @@ -33,6 +29,11 @@ #define PROPERTY_TO_OPTIONAL(COMMAND, PROPERTY) \ COMMAND.has_##PROPERTY() ? util::make_optional(COMMAND.PROPERTY()) : util::nullopt; +#define REGISTER_FIELD(name, type, nullable) \ + const auto& name = field(ARROW_STRINGIFY(name), type, nullable); + +#define REGISTER_FIELD_NOT_NULL(name, type) REGISTER_FIELD(name, type, false) + namespace arrow { namespace flight { namespace sql { @@ -98,13 +99,24 @@ arrow::Result ParseCommandGetPrimaryKeys( return result; } -arrow::Result ParseCommandGetSqlInfo(const google::protobuf::Any& any) { +arrow::Result ParseCommandGetSqlInfo( + const google::protobuf::Any& any, + const sql_info_id_to_result_t& 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; } @@ -263,7 +275,8 @@ Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, } else if (any.Is()) { return GetFlightInfoTableTypes(context, request, info); } else if (any.Is()) { - ARROW_ASSIGN_OR_RAISE(GetSqlInfo internal_command, ParseCommandGetSqlInfo(any)); + ARROW_ASSIGN_OR_RAISE(GetSqlInfo internal_command, + ParseCommandGetSqlInfo(any, sql_info_id_to_result_)); return GetFlightInfoSqlInfo(internal_command, context, request, info); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetPrimaryKeys internal_command, @@ -313,7 +326,8 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket } else if (any.Is()) { return DoGetTableTypes(context, stream); } else if (any.Is()) { - ARROW_ASSIGN_OR_RAISE(GetSqlInfo internal_command, ParseCommandGetSqlInfo(any)); + ARROW_ASSIGN_OR_RAISE(GetSqlInfo internal_command, + ParseCommandGetSqlInfo(any, sql_info_id_to_result_)); return DoGetSqlInfo(internal_command, context, stream); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetPrimaryKeys internal_command, @@ -479,13 +493,88 @@ Status FlightSqlServerBase::GetFlightInfoSqlInfo(const GetSqlInfo& command, const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) { - return Status::NotImplemented("GetFlightInfoSqlInfo not implemented"); + assert(info != nullptr); + + if (sql_info_id_to_result_.empty()) { + return Status::NotImplemented("GetFlightInfoSqlInfo not implemented"); + } + + std::vector endpoints{FlightEndpoint{{descriptor.cmd}, {}}}; + ARROW_ASSIGN_OR_RAISE(auto result, FlightInfo::Make(*SqlSchema::GetSqlInfoSchema(), + descriptor, endpoints, -1, -1)) + + *info = std::unique_ptr(new FlightInfo(result)); + + return Status::OK(); +} + +void FlightSqlServerBase::RegisterSqlInfo(int32_t id, const SqlInfoResult& result) { + sql_info_id_to_result_[id] = result; } Status FlightSqlServerBase::DoGetSqlInfo(const GetSqlInfo& command, const ServerCallContext& context, std::unique_ptr* result) { - return Status::NotImplemented("DoGetSqlInfo not implemented"); + assert(result != nullptr); + + MemoryPool* memory_pool = default_memory_pool(); + UInt32Builder name_field_builder(memory_pool); + const std::shared_ptr string_value_field_builder( + new StringBuilder(memory_pool)); + const std::shared_ptr bool_value_field_builder( + new BooleanBuilder(memory_pool)); + const std::shared_ptr bigint_value_field_builder( + new Int64Builder(memory_pool)); + const std::shared_ptr int32_bitmask_field_builder( + new Int32Builder(memory_pool)); + const std::shared_ptr string_list_inner_string_field_builder( + new StringBuilder(memory_pool)); + const std::shared_ptr string_list_field_builder( + new ListBuilder(memory_pool, string_list_inner_string_field_builder)); + const std::shared_ptr int32_to_int32_list_key_builder( + new Int32Builder(memory_pool)); + const std::shared_ptr int32_to_int32_list_elements_in_value_list( + new Int32Builder(memory_pool)); + const std::shared_ptr int32_to_int32_list_value_builder( + new ListBuilder(memory_pool, int32_to_int32_list_elements_in_value_list)); + const std::shared_ptr int32_to_int32_list_map_field_builder(new MapBuilder( + memory_pool, int32_to_int32_list_key_builder, int32_to_int32_list_value_builder)); + std::vector> value_field_builders{ + string_value_field_builder, bool_value_field_builder, + bigint_value_field_builder, int32_bitmask_field_builder, + string_list_field_builder, int32_to_int32_list_map_field_builder}; + const std::shared_ptr& schema = SqlSchema::GetSqlInfoSchema(); + const auto& union_type = schema->fields()[1]->type(); // expected union type. + DenseUnionBuilder value_field_builder(memory_pool, value_field_builders, union_type); + internal::SqlInfoResultAppender sql_info_result_appender(value_field_builder); + + // 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) { + if (!sql_info_id_to_result_.count(info)) { + return Status::KeyError( + "Attempt to fetch unregistered/unsupported SqlInfo option: " + + std::to_string(info)); + } + ARROW_RETURN_NOT_OK(name_field_builder.Append(info)); + ARROW_RETURN_NOT_OK( + boost::apply_visitor(sql_info_result_appender, sql_info_id_to_result_.at(info))); + } + + 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(schema, row_count, {name, value}); + ARROW_ASSIGN_OR_RAISE(const auto reader, RecordBatchReader::Make({batch})); + *result = std::unique_ptr(new RecordBatchStream(reader)); + return Status::OK(); } Status FlightSqlServerBase::GetFlightInfoSchemas(const GetSchemas& command, @@ -660,6 +749,20 @@ std::shared_ptr SqlSchema::GetCrossReferenceSchema() { return GetImportedExportedKeysAndCrossReferenceSchema(); } +std::shared_ptr SqlSchema::GetSqlInfoSchema() { + REGISTER_FIELD_NOT_NULL(name, uint32()) + REGISTER_FIELD_NOT_NULL(string_value, utf8()) + REGISTER_FIELD_NOT_NULL(bool_value, boolean()) + REGISTER_FIELD_NOT_NULL(bigint_value, int64()) + REGISTER_FIELD_NOT_NULL(int32_bitmask, int32()) + REGISTER_FIELD_NOT_NULL(string_list, list(utf8())) + REGISTER_FIELD_NOT_NULL(int32_to_int32_list_map, map(int32(), list(int32()))) + const FieldVector value_fields{string_value, bool_value, bigint_value, + int32_bitmask, string_list, int32_to_int32_list_map}; + REGISTER_FIELD_NOT_NULL(value, dense_union(value_fields)) + return arrow::schema({name, value}); +} + } // namespace sql } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index 031e1135152..b1099dac512 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -20,15 +20,24 @@ #pragma once +#include #include #include +#include #include "arrow/flight/flight_sql/example/sqlite_statement.h" #include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" #include "arrow/flight/flight_sql/server.h" +#include "arrow/flight/flight_sql/sql_info_util.h" #include "arrow/flight/server.h" #include "arrow/util/optional.h" +using string_list_t = std::vector; +using int32_to_int32_list_t = std::unordered_map>; +using SqlInfoResult = boost::variant; +using sql_info_id_to_result_t = std::unordered_map; + namespace arrow { namespace flight { namespace sql { @@ -54,7 +63,7 @@ struct PreparedStatementUpdate { }; struct GetSqlInfo { - // TODO: To be implemented. + std::vector info; }; struct GetSchemas { @@ -112,6 +121,9 @@ struct ActionCreatePreparedStatementResult { }; class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { + private: + sql_info_id_to_result_t sql_info_id_to_result_; + public: Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, std::unique_ptr* info) override; @@ -431,6 +443,11 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { protected: static std::string CreateStatementQueryTicket(const std::string& statement_handle); + + /// \brief Registers a new SqlInfo result. + /// \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. @@ -474,6 +491,10 @@ class ARROW_EXPORT SqlSchema { /// \brief Gets the Schema used on GetCrossReference response. /// \return The default schema template. static std::shared_ptr GetCrossReferenceSchema(); + + /// \brief Gets the Schema used on GetSqlInfo response. + /// \return The default schema template. + static std::shared_ptr GetSqlInfoSchema(); }; } // namespace sql } // namespace flight diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index ff22e9088c1..3dd78007893 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -38,6 +38,87 @@ namespace arrow { namespace flight { namespace sql { +/// \brief Auxiliary boost::variant visitor used to assert that GetSqlInfo's values are +/// correctly placed on its DenseUnionArray +class SqlInfoDenseUnionValidator : public boost::static_visitor { + 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 = dynamic_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 = dynamic_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 = dynamic_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 = dynamic_cast(*data.value); + ASSERT_EQ(int32_bitmask, scalar.value); + } + + /// \brief Asserts that the current DenseUnionScalar equals to given string list + void operator()(const string_list_t& string_list) const { + const auto& array = dynamic_cast( + *(dynamic_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 int32_to_int32_list_t& int32_to_int32_list) const { + const auto& struct_array = dynamic_cast( + *dynamic_cast(*data.value).value); + const auto& keys = dynamic_cast(*struct_array.field(0)); + const auto& values = dynamic_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 = dynamic_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 = dynamic_cast(*list_item_scalar).value; + ASSERT_EQ(expected_int32_list[j], list_item); + } + } + } + + explicit SqlInfoDenseUnionValidator(const DenseUnionScalar& data) : data(data) {} +}; + std::unique_ptr server; std::unique_ptr sql_client; @@ -652,6 +733,40 @@ TEST(TestFlightSqlServer, TestCommandGetCrossReference) { AssertTablesEqual(*expected_table, *table); } +TEST(TestFlightSqlServer, TestCommandGetSqlInfo) { + const auto& sql_info_expected_results = sql::example::GetSqlInfoIdToResult(); + 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]); + boost::apply_visitor(validator, expected_result); + } + } +} + auto env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); } // namespace sql diff --git a/cpp/src/arrow/flight/flight_sql/sql_info_util.cc b/cpp/src/arrow/flight/flight_sql/sql_info_util.cc new file mode 100644 index 00000000000..e77ce0bf64f --- /dev/null +++ b/cpp/src/arrow/flight/flight_sql/sql_info_util.cc @@ -0,0 +1,113 @@ +// 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 + +#define PROPERTY_TO_OPTIONAL(COMMAND, PROPERTY) \ + COMMAND.has_##PROPERTY() ? util::make_optional(COMMAND.PROPERTY()) : util::nullopt; + +#define REGISTER_FIELD(name, type, nullable) \ + const auto& name = field(ARROW_STRINGIFY(name), type, nullable); + +#define REGISTER_FIELD_NOT_NULL(name, type) REGISTER_FIELD(name, type, false) + +namespace arrow { +namespace flight { +namespace sql { +namespace internal { + +Status SqlInfoResultAppender::operator()(const std::string& value) { + const int8_t type_index = STRING_VALUE_INDEX; + ARROW_RETURN_NOT_OK(value_builder_.Append(type_index)); + auto* string_value_builder = + reinterpret_cast(value_builder_.child(type_index)); + ARROW_RETURN_NOT_OK(string_value_builder->Append(value)); + return Status::OK(); +} + +Status SqlInfoResultAppender::operator()(const bool value) { + const int8_t type_index = BOOL_VALUE_INDEX; + ARROW_RETURN_NOT_OK(value_builder_.Append(type_index)); + auto* bool_value_builder = + reinterpret_cast(value_builder_.child(type_index)); + ARROW_RETURN_NOT_OK(bool_value_builder->Append(value)); + return Status::OK(); +} + +Status SqlInfoResultAppender::operator()(const int64_t value) { + const int8_t type_index = BIGINT_VALUE_INDEX; + ARROW_RETURN_NOT_OK(value_builder_.Append(type_index)); + auto* bigint_value_builder = + reinterpret_cast(value_builder_.child(type_index)); + ARROW_RETURN_NOT_OK(bigint_value_builder->Append(value)); + return Status::OK(); +} + +Status SqlInfoResultAppender::operator()(const int32_t value) { + const int8_t type_index = INT32_BITMASK_INDEX; + ARROW_RETURN_NOT_OK(value_builder_.Append(type_index)); + auto* int32_bitmask_builder = + reinterpret_cast(value_builder_.child(type_index)); + ARROW_RETURN_NOT_OK(int32_bitmask_builder->Append(value)); + return Status::OK(); +} + +Status SqlInfoResultAppender::operator()(const string_list_t& value) { + const int8_t type_index = STRING_LIST_INDEX; + ARROW_RETURN_NOT_OK(value_builder_.Append(type_index)); + auto* string_list_builder = + reinterpret_cast(value_builder_.child(type_index)); + 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 int32_to_int32_list_t& value) { + const int8_t type_index = INT32_TO_INT32_LIST_INDEX; + ARROW_RETURN_NOT_OK(value_builder_.Append(type_index)); + auto* int32_to_int32_list_builder = + reinterpret_cast(value_builder_.child(type_index)); + 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) {} + +} // namespace internal +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight_sql/sql_info_util.h b/cpp/src/arrow/flight/flight_sql/sql_info_util.h new file mode 100644 index 00000000000..1e6464f25af --- /dev/null +++ b/cpp/src/arrow/flight/flight_sql/sql_info_util.h @@ -0,0 +1,84 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// Interfaces to use for defining Flight RPC servers. API should be considered +// experimental for now + +#pragma once + +#include + +using string_list_t = std::vector; +using int32_to_int32_list_t = std::unordered_map>; +using SqlInfoResult = boost::variant; +using sql_info_id_to_result_t = std::unordered_map; + +namespace arrow { +namespace flight { +namespace sql { +namespace internal { + +/// \brief Auxiliary class used to populate GetSqlInfo's DenseUnionArray with different +/// data types. +class SqlInfoResultAppender : public boost::static_visitor { + public: + /// \brief Appends a string to the DenseUnionBuilder. + /// \param[in] value Value to be appended. + Status operator()(const std::string& value); + + /// \brief Appends a bool to the DenseUnionBuilder. + /// \param[in] value Value to be appended. + Status operator()(bool value); + + /// \brief Appends a int64_t to the DenseUnionBuilder. + /// \param[in] value Value to be appended. + Status operator()(int64_t value); + + /// \brief Appends a int64_t to the DenseUnionBuilder. + /// \param[in] value Value to be appended. + Status operator()(int32_t value); + + /// \brief Appends a string list to the DenseUnionBuilder. + /// \param[in] value Value to be appended. + Status operator()(const string_list_t& value); + + /// \brief Appends a int32 to int32 list map to the DenseUnionBuilder. + /// \param[in] value Value to be appended. + Status operator()(const int32_to_int32_list_t& value); + + /// \brief Creates a boost::variant visitor that appends data to given + /// DenseUnionBuilder. \param[in] value_builder DenseUnionBuilder to append data to. + explicit SqlInfoResultAppender(DenseUnionBuilder& value_builder); + + private: + DenseUnionBuilder& value_builder_; + + enum : int8_t { + STRING_VALUE_INDEX = 0, + BOOL_VALUE_INDEX = 1, + BIGINT_VALUE_INDEX = 2, + INT32_BITMASK_INDEX = 3, + STRING_LIST_INDEX = 4, + INT32_TO_INT32_LIST_INDEX = 5 + }; +}; + +} // namespace internal +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc index 91b57f99319..88b81667b52 100644 --- a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc @@ -171,6 +171,8 @@ Status RunMain() { } else if (FLAGS_command == "GetPrimaryKeys") { ARROW_ASSIGN_OR_RAISE(info, sql_client.GetPrimaryKeys(call_options, &FLAGS_catalog, &FLAGS_schema, FLAGS_table)); + } else if (FLAGS_command == "GetSqlInfo") { + ARROW_ASSIGN_OR_RAISE(info, sql_client.GetSqlInfo(call_options, {})); } if (info != NULLPTR && diff --git a/format/FlightSql.proto b/format/FlightSql.proto index 123558c6aa5..759654f49a7 100644 --- a/format/FlightSql.proto +++ b/format/FlightSql.proto @@ -37,6 +37,7 @@ package arrow.flight.protocol.sql; * 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. */ @@ -203,8 +204,13 @@ enum SqlInfo { */ SQL_NULL_PLUS_NULL_IS_NULL = 516; - // Retrieves a UTF-8 string list with values of the supported conversion functions. - SQL_SUPPORTED_CONVERSION_FUNCTIONS = 517; + /* + * 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, @@ -326,12 +332,12 @@ enum SqlInfo { SQL_SUPPORTS_INTEGRITY_ENHANCEMENT_FACILITY = 527; /* - * Retrieves the support level for SQL JOINs. + * 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.SqlJoinsSupportLevel`. + * `arrow.flight.protocol.sql.SqlOuterJoinsSupportLevel`. */ - SQL_JOINS_SUPPORT_LEVEL = 528; + SQL_OUTER_JOINS_SUPPORT_LEVEL = 528; // Retrieves a UTF-8 string with the preferred term for "schema". SQL_SCHEMA_TERM = 529; @@ -777,9 +783,9 @@ enum SupportedAnsi92SqlGrammarLevel { ANSI92_FULL_SQL = 2; } -enum SqlJoinsSupportLevel { +enum SqlOuterJoinsSupportLevel { SQL_JOINS_UNSUPPORTED = 0; - SQL_LIMITED_JOINS = 1; + SQL_LIMITED_OUTER_JOINS = 1; SQL_FULL_OUTER_JOINS = 2; } @@ -838,6 +844,29 @@ enum SqlSupportedResultSetConcurrency { 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. * Used in the command member of FlightDescriptor for the following RPC calls: From 776d739476793cd47f70f156e144f966ec338126 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Thu, 11 Nov 2021 11:25:45 -0300 Subject: [PATCH 205/237] Fix method docs errors on server.h and sqlite example --- .../flight_sql/example/sqlite_tables_schema_batch_reader.h | 2 +- cpp/src/arrow/flight/flight_sql/server.h | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h index e4fcb6680c7..dce725e7a2b 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h @@ -40,7 +40,7 @@ class SqliteTablesWithSchemaBatchReader : public RecordBatchReader { public: /// Constructor for SqliteTablesWithSchemaBatchReader class /// \param reader an shared_ptr from a SqliteStatementBatchReader. - /// \param query SQL query that originated reader's data. + /// \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, diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index b1099dac512..08702a2e37a 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -166,7 +166,6 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \brief Gets a FlightDataStream containing the query results. /// \param[in] command The StatementQueryTicket containing the statement handle. /// \param[in] context Per-call context. - /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] result The FlightDataStream containing the results. /// \return Status. virtual Status DoGetStatement(const StatementQueryTicket& command, @@ -190,7 +189,6 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \param[in] command The PreparedStatementQuery object containing the /// prepared statement handle. /// \param[in] context Per-call context. - /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] result The FlightDataStream containing the results. /// \return Status. virtual Status DoGetPreparedStatement(const PreparedStatementQuery& command, From 9e352e3a77231942562964c968926d32e3d4cea2 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 10 Nov 2021 17:18:48 -0300 Subject: [PATCH 206/237] Change CreateStatementQueryTicket to a free function --- .../flight/flight_sql/example/sqlite_server.cc | 3 ++- cpp/src/arrow/flight/flight_sql/server.cc | 14 ++++++++++++++ cpp/src/arrow/flight/flight_sql/server.h | 8 ++++++++ 3 files changed, 24 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc index 4b7e40aee4f..945b7b68dfd 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc @@ -264,7 +264,8 @@ Status SQLiteFlightSqlServer::GetFlightInfoStatement(const StatementQuery& comma std::shared_ptr schema; ARROW_RETURN_NOT_OK(statement->GetSchema(&schema)); - std::string ticket_string = CreateStatementQueryTicket(query); + std::string ticket_string; + ARROW_RETURN_NOT_OK(CreateStatementQueryTicket(query, &ticket_string)); std::vector endpoints{FlightEndpoint{{ticket_string}, {}}}; ARROW_ASSIGN_OR_RAISE(auto result, FlightInfo::Make(*schema, descriptor, endpoints, -1, -1)) diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index 6690c221e79..7b737ef19f0 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -40,6 +40,20 @@ namespace sql { namespace pb = arrow::flight::protocol; +Status CreateStatementQueryTicket(const std::string& statement_handle, + std::string* ticket_string) { + protocol::sql::TicketStatementQuery ticket_statement_query; + ticket_statement_query.set_statement_handle(statement_handle); + + google::protobuf::Any ticket; + ticket.PackFrom(ticket_statement_query); + + if (!ticket.SerializeToString(ticket_string)) { + return Status::IOError("Invalid ticket."); + } + return Status::OK(); +} + arrow::Result ParseCommandGetCrossReference( const google::protobuf::Any& any) { pb::sql::CommandGetCrossReference command; diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index 08702a2e37a..c5497b1cee9 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -120,6 +120,14 @@ struct ActionCreatePreparedStatementResult { std::string prepared_statement_handle; }; +/// \brief A free function responsible to create a ticket for the statement operation. +/// It is used by the server implementation. +/// \param[in] statement_handle The statement handle that will originate the ticket. +/// \param[out] ticket_string The parsed ticket as a string. +/// \return Status +Status CreateStatementQueryTicket(const std::string &statement_handle, + std::string *ticket_string); + class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { private: sql_info_id_to_result_t sql_info_id_to_result_; From 8ececc8b17431fc50b40d2a045f86adc1d41974b Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 10 Nov 2021 17:19:21 -0300 Subject: [PATCH 207/237] Remove old use of CreateStatementQueryTicket --- cpp/src/arrow/flight/flight_sql/server.cc | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index 7b737ef19f0..4129bdd3b92 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -250,18 +250,6 @@ ParseActionClosePreparedStatementRequest(const google::protobuf::Any& any) { return result; } -std::string FlightSqlServerBase::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); - - const std::string& ticket_string = ticket.SerializeAsString(); - return ticket_string; -} - Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, std::unique_ptr* info) { From 26f0c194fce55951d1c4af96e69b3cd0862e046d Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 10 Nov 2021 17:20:35 -0300 Subject: [PATCH 208/237] Make parameter ordering consistent --- .../flight_sql/example/sqlite_server.cc | 56 ++--- .../flight/flight_sql/example/sqlite_server.h | 70 +++--- cpp/src/arrow/flight/flight_sql/server.cc | 133 ++++++------ cpp/src/arrow/flight/flight_sql/server.h | 204 +++++++++--------- 4 files changed, 234 insertions(+), 229 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc index 945b7b68dfd..e89c7ff31cf 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc @@ -252,8 +252,8 @@ Status GetFlightInfoForCommand(const FlightDescriptor& descriptor, return Status::OK(); } -Status SQLiteFlightSqlServer::GetFlightInfoStatement(const StatementQuery& command, - const ServerCallContext& context, +Status SQLiteFlightSqlServer::GetFlightInfoStatement(const ServerCallContext& context, + const StatementQuery& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { const std::string& query = command.query; @@ -275,8 +275,8 @@ Status SQLiteFlightSqlServer::GetFlightInfoStatement(const StatementQuery& comma return Status::OK(); } -Status SQLiteFlightSqlServer::DoGetStatement(const StatementQueryTicket& command, - const ServerCallContext& context, +Status SQLiteFlightSqlServer::DoGetStatement(const ServerCallContext& context, + const StatementQueryTicket& command, std::unique_ptr* result) { const std::string& sql = command.statement_handle; @@ -314,15 +314,15 @@ Status SQLiteFlightSqlServer::DoGetCatalogs(const ServerCallContext& context, return Status::OK(); } -Status SQLiteFlightSqlServer::GetFlightInfoSchemas(const GetSchemas& command, - const ServerCallContext& context, +Status SQLiteFlightSqlServer::GetFlightInfoSchemas(const ServerCallContext& context, + const GetSchemas& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetSchemasSchema()); } -Status SQLiteFlightSqlServer::DoGetSchemas(const GetSchemas& command, - const ServerCallContext& context, +Status SQLiteFlightSqlServer::DoGetSchemas(const ServerCallContext& context, + const GetSchemas& command, std::unique_ptr* result) { // As SQLite doesn't support schemas, this will return an empty record batch. @@ -341,8 +341,8 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const GetSchemas& command, return Status::OK(); } -Status SQLiteFlightSqlServer::GetFlightInfoTables(const GetTables& command, - const ServerCallContext& context, +Status SQLiteFlightSqlServer::GetFlightInfoTables(const ServerCallContext& context, + const GetTables& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { std::vector endpoints{FlightEndpoint{{descriptor.cmd}, {}}}; @@ -359,8 +359,8 @@ Status SQLiteFlightSqlServer::GetFlightInfoTables(const GetTables& command, return Status::OK(); } -Status SQLiteFlightSqlServer::DoGetTables(const GetTables& command, - const ServerCallContext& context, +Status SQLiteFlightSqlServer::DoGetTables(const ServerCallContext& context, + const GetTables& command, std::unique_ptr* result) { std::string query = PrepareQueryForGetTables(command); @@ -384,7 +384,7 @@ Status SQLiteFlightSqlServer::DoGetTables(const GetTables& command, } arrow::Result SQLiteFlightSqlServer::DoPutCommandStatementUpdate( - const StatementUpdate& command, const ServerCallContext& context, + const ServerCallContext& context, const StatementUpdate& command, std::unique_ptr& reader) { const std::string& sql = command.query; @@ -399,8 +399,8 @@ arrow::Result SQLiteFlightSqlServer::DoPutCommandStatementUpdate( arrow::Result SQLiteFlightSqlServer::CreatePreparedStatement( - const ActionCreatePreparedStatementRequest& request, - const ServerCallContext& context) { + const ServerCallContext& context, + const ActionCreatePreparedStatementRequest& request) { std::shared_ptr statement; ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, request.query)); boost::uuids::uuid uuid = uuid_generator_(); @@ -441,7 +441,7 @@ SQLiteFlightSqlServer::CreatePreparedStatement( } Status SQLiteFlightSqlServer::ClosePreparedStatement( - const ActionClosePreparedStatementRequest& request, const ServerCallContext& context, + const ServerCallContext& context, const ActionClosePreparedStatementRequest& request, std::unique_ptr* result) { const std::string& prepared_statement_handle = request.prepared_statement_handle; const auto& uuid = boost::lexical_cast(prepared_statement_handle); @@ -459,7 +459,7 @@ Status SQLiteFlightSqlServer::ClosePreparedStatement( } Status SQLiteFlightSqlServer::GetFlightInfoPreparedStatement( - const PreparedStatementQuery& command, const ServerCallContext& context, + const ServerCallContext& context, const PreparedStatementQuery& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { const std::string& prepared_statement_handle = command.prepared_statement_handle; const auto& uuid = boost::lexical_cast(prepared_statement_handle); @@ -478,7 +478,7 @@ Status SQLiteFlightSqlServer::GetFlightInfoPreparedStatement( } Status SQLiteFlightSqlServer::DoGetPreparedStatement( - const PreparedStatementQuery& command, const ServerCallContext& context, + const ServerCallContext& context, const PreparedStatementQuery& command, std::unique_ptr* result) { const std::string& prepared_statement_handle = command.prepared_statement_handle; const auto& uuid = boost::lexical_cast(prepared_statement_handle); @@ -499,7 +499,7 @@ Status SQLiteFlightSqlServer::DoGetPreparedStatement( } Status SQLiteFlightSqlServer::DoPutPreparedStatementQuery( - const PreparedStatementQuery& command, const ServerCallContext& context, + 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_, @@ -512,7 +512,7 @@ Status SQLiteFlightSqlServer::DoPutPreparedStatementQuery( } arrow::Result SQLiteFlightSqlServer::DoPutPreparedStatementUpdate( - const PreparedStatementUpdate& command, const ServerCallContext& context, + 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_, @@ -541,13 +541,13 @@ Status SQLiteFlightSqlServer::DoGetTableTypes(const ServerCallContext& context, } Status SQLiteFlightSqlServer::GetFlightInfoPrimaryKeys( - const GetPrimaryKeys& command, const ServerCallContext& context, + const ServerCallContext& context, const GetPrimaryKeys& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetPrimaryKeysSchema()); } Status SQLiteFlightSqlServer::DoGetPrimaryKeys( - const GetPrimaryKeys& command, const ServerCallContext& context, + const ServerCallContext& context, const GetPrimaryKeys& command, std::unique_ptr* result) { std::stringstream table_query; @@ -608,13 +608,13 @@ std::string PrepareQueryForGetImportedOrExportedKeys(const std::string& filter) } Status SQLiteFlightSqlServer::GetFlightInfoImportedKeys( - const GetImportedKeys& command, const ServerCallContext& context, + const ServerCallContext& context, const GetImportedKeys& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetImportedKeysSchema()); } Status SQLiteFlightSqlServer::DoGetImportedKeys( - const GetImportedKeys& command, const ServerCallContext& context, + const ServerCallContext& context, const GetImportedKeys& command, std::unique_ptr* result) { std::string filter = "fk_table_name = '" + command.table + "'"; if (command.catalog.has_value()) { @@ -629,13 +629,13 @@ Status SQLiteFlightSqlServer::DoGetImportedKeys( } Status SQLiteFlightSqlServer::GetFlightInfoExportedKeys( - const GetExportedKeys& command, const ServerCallContext& context, + const ServerCallContext& context, const GetExportedKeys& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetExportedKeysSchema()); } Status SQLiteFlightSqlServer::DoGetExportedKeys( - const GetExportedKeys& command, const ServerCallContext& context, + const ServerCallContext& context, const GetExportedKeys& command, std::unique_ptr* result) { std::string filter = "pk_table_name = '" + command.table + "'"; if (command.catalog.has_value()) { @@ -650,13 +650,13 @@ Status SQLiteFlightSqlServer::DoGetExportedKeys( } Status SQLiteFlightSqlServer::GetFlightInfoCrossReference( - const GetCrossReference& command, const ServerCallContext& context, + const ServerCallContext& context, const GetCrossReference& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetCrossReferenceSchema()); } Status SQLiteFlightSqlServer::DoGetCrossReference( - const GetCrossReference& command, const ServerCallContext& context, + const ServerCallContext& context, const GetCrossReference& command, std::unique_ptr* result) { std::string filter = "pk_table_name = '" + command.pk_table + "'"; if (command.pk_catalog.has_value()) { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h index 298268e3708..710316e778f 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h @@ -252,87 +252,87 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { /// SQLite database. Status ExecuteSql(const std::string& sql); - Status GetFlightInfoStatement(const StatementQuery& command, - const ServerCallContext& context, + Status GetFlightInfoStatement(const ServerCallContext& context, + const StatementQuery& command, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetStatement(const StatementQueryTicket& command, - const ServerCallContext& context, + Status DoGetStatement(const ServerCallContext& context, + const StatementQueryTicket& command, std::unique_ptr* result) override; Status GetFlightInfoCatalogs(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) override; Status DoGetCatalogs(const ServerCallContext& context, std::unique_ptr* result) override; - Status GetFlightInfoSchemas(const GetSchemas& command, const ServerCallContext& context, + Status GetFlightInfoSchemas(const ServerCallContext& context, const GetSchemas& command, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetSchemas(const GetSchemas& command, const ServerCallContext& context, + Status DoGetSchemas(const ServerCallContext& context, const GetSchemas& command, std::unique_ptr* result) override; arrow::Result DoPutCommandStatementUpdate( - const StatementUpdate& update, const ServerCallContext& context, + const ServerCallContext& context, const StatementUpdate& update, std::unique_ptr& reader) override; arrow::Result CreatePreparedStatement( - const ActionCreatePreparedStatementRequest& request, - const ServerCallContext& context) override; - Status ClosePreparedStatement(const ActionClosePreparedStatementRequest& request, - const ServerCallContext& context, + const ServerCallContext& context, + const ActionCreatePreparedStatementRequest& request) override; + Status ClosePreparedStatement(const ServerCallContext& context, + const ActionClosePreparedStatementRequest& request, std::unique_ptr* result) override; - Status GetFlightInfoPreparedStatement(const PreparedStatementQuery& command, - const ServerCallContext& context, + Status GetFlightInfoPreparedStatement(const ServerCallContext& context, + const PreparedStatementQuery& command, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetPreparedStatement(const PreparedStatementQuery& command, - const ServerCallContext& context, + Status DoGetPreparedStatement(const ServerCallContext& context, + const PreparedStatementQuery& command, std::unique_ptr* result) override; - Status DoPutPreparedStatementQuery(const PreparedStatementQuery& command, - const ServerCallContext& context, + Status DoPutPreparedStatementQuery(const ServerCallContext& context, + const PreparedStatementQuery& command, FlightMessageReader* reader, FlightMetadataWriter* writer) override; arrow::Result DoPutPreparedStatementUpdate( - const PreparedStatementUpdate& command, const ServerCallContext& context, + const ServerCallContext& context, const PreparedStatementUpdate& command, FlightMessageReader* reader) override; - Status GetFlightInfoTables(const GetTables& command, const ServerCallContext& context, + Status GetFlightInfoTables(const ServerCallContext& context, const GetTables& command, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetTables(const GetTables& command, const ServerCallContext& context, + Status DoGetTables(const ServerCallContext& context, const GetTables& command, std::unique_ptr* result) override; Status GetFlightInfoTableTypes(const ServerCallContext& context, const FlightDescriptor& descriptor, std::unique_ptr* info) override; Status DoGetTableTypes(const ServerCallContext& context, std::unique_ptr* result) override; - Status GetFlightInfoImportedKeys(const GetImportedKeys& command, - const ServerCallContext& context, + Status GetFlightInfoImportedKeys(const ServerCallContext& context, + const GetImportedKeys& command, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetImportedKeys(const GetImportedKeys& command, - const ServerCallContext& context, + Status DoGetImportedKeys(const ServerCallContext& context, + const GetImportedKeys& command, std::unique_ptr* result) override; - Status GetFlightInfoExportedKeys(const GetExportedKeys& command, - const ServerCallContext& context, + Status GetFlightInfoExportedKeys(const ServerCallContext& context, + const GetExportedKeys& command, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetExportedKeys(const GetExportedKeys& command, - const ServerCallContext& context, + Status DoGetExportedKeys(const ServerCallContext& context, + const GetExportedKeys& command, std::unique_ptr* result) override; - Status GetFlightInfoCrossReference(const GetCrossReference& command, - const ServerCallContext& context, + Status GetFlightInfoCrossReference(const ServerCallContext& context, + const GetCrossReference& command, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetCrossReference(const GetCrossReference& command, - const ServerCallContext& context, + Status DoGetCrossReference(const ServerCallContext& context, + const GetCrossReference& command, std::unique_ptr* result) override; - Status GetFlightInfoPrimaryKeys(const GetPrimaryKeys& command, - const ServerCallContext& context, + Status GetFlightInfoPrimaryKeys(const ServerCallContext& context, + const GetPrimaryKeys& command, const FlightDescriptor& descriptor, std::unique_ptr* info) override; - Status DoGetPrimaryKeys(const GetPrimaryKeys& command, const ServerCallContext& context, + Status DoGetPrimaryKeys(const ServerCallContext& context, const GetPrimaryKeys& command, std::unique_ptr* result) override; private: diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index 4129bdd3b92..85504b79f1d 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -261,41 +261,41 @@ Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, if (any.Is()) { ARROW_ASSIGN_OR_RAISE(StatementQuery internal_command, ParseCommandStatementQuery(any)); - return GetFlightInfoStatement(internal_command, context, request, info); + return GetFlightInfoStatement(context, internal_command, request, info); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command, ParseCommandPreparedStatementQuery(any)); - return GetFlightInfoPreparedStatement(internal_command, context, request, info); + return GetFlightInfoPreparedStatement(context, internal_command, request, info); } else if (any.Is()) { return GetFlightInfoCatalogs(context, request, info); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetSchemas internal_command, ParseCommandGetSchemas(any)); - return GetFlightInfoSchemas(internal_command, context, request, info); + return GetFlightInfoSchemas(context, internal_command, request, info); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetTables command, ParseCommandGetTables(any)); - return GetFlightInfoTables(command, context, request, info); + return GetFlightInfoTables(context, command, request, info); } else if (any.Is()) { return GetFlightInfoTableTypes(context, request, info); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetSqlInfo internal_command, ParseCommandGetSqlInfo(any, sql_info_id_to_result_)); - return GetFlightInfoSqlInfo(internal_command, context, request, info); + return GetFlightInfoSqlInfo(context, internal_command, request, info); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetPrimaryKeys internal_command, ParseCommandGetPrimaryKeys(any)); - return GetFlightInfoPrimaryKeys(internal_command, context, request, info); + return GetFlightInfoPrimaryKeys(context, internal_command, request, info); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetExportedKeys internal_command, ParseCommandGetExportedKeys(any)); - return GetFlightInfoExportedKeys(internal_command, context, request, info); + return GetFlightInfoExportedKeys(context, internal_command, request, info); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetImportedKeys internal_command, ParseCommandGetImportedKeys(any)); - return GetFlightInfoImportedKeys(internal_command, context, request, info); + return GetFlightInfoImportedKeys(context, internal_command, request, info); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetCrossReference internal_command, ParseCommandGetCrossReference(any)); - return GetFlightInfoCrossReference(internal_command, context, request, info); + return GetFlightInfoCrossReference(context, internal_command, request, info); } return Status::Invalid("The defined request is invalid."); @@ -312,41 +312,41 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket if (any.Is()) { ARROW_ASSIGN_OR_RAISE(StatementQueryTicket command, ParseStatementQueryTicket(any)); - return DoGetStatement(command, context, stream); + return DoGetStatement(context, command, stream); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command, ParseCommandPreparedStatementQuery(any)); - return DoGetPreparedStatement(internal_command, context, stream); + return DoGetPreparedStatement(context, internal_command, stream); } else if (any.Is()) { return DoGetCatalogs(context, stream); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetSchemas internal_command, ParseCommandGetSchemas(any)); - return DoGetSchemas(internal_command, context, stream); + return DoGetSchemas(context, internal_command, stream); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetTables command, ParseCommandGetTables(any)); - return DoGetTables(command, context, stream); + return DoGetTables(context, command, stream); } else if (any.Is()) { return DoGetTableTypes(context, stream); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetSqlInfo internal_command, ParseCommandGetSqlInfo(any, sql_info_id_to_result_)); - return DoGetSqlInfo(internal_command, context, stream); + return DoGetSqlInfo(context, internal_command, stream); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetPrimaryKeys internal_command, ParseCommandGetPrimaryKeys(any)); - return DoGetPrimaryKeys(internal_command, context, stream); + return DoGetPrimaryKeys(context, internal_command, stream); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetExportedKeys internal_command, ParseCommandGetExportedKeys(any)); - return DoGetExportedKeys(internal_command, context, stream); + return DoGetExportedKeys(context, internal_command, stream); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetImportedKeys internal_command, ParseCommandGetImportedKeys(any)); - return DoGetImportedKeys(internal_command, context, stream); + return DoGetImportedKeys(context, internal_command, stream); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetCrossReference internal_command, ParseCommandGetCrossReference(any)); - return DoGetCrossReference(internal_command, context, stream); + return DoGetCrossReference(context, internal_command, stream); } return Status::Invalid("The defined request is invalid."); @@ -366,7 +366,7 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, ARROW_ASSIGN_OR_RAISE(StatementUpdate internal_command, ParseCommandStatementUpdate(any)); ARROW_ASSIGN_OR_RAISE(auto record_count, - DoPutCommandStatementUpdate(internal_command, context, reader)) + DoPutCommandStatementUpdate(context, internal_command, reader)) pb::sql::DoPutUpdateResult result; result.set_record_count(record_count); @@ -379,13 +379,13 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command, ParseCommandPreparedStatementQuery(any)); - return DoPutPreparedStatementQuery(internal_command, context, reader.get(), + 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( - internal_command, context, reader.get())) + context, internal_command, reader.get())) pb::sql::DoPutUpdateResult result; result.set_record_count(record_count); @@ -419,7 +419,7 @@ Status FlightSqlServerBase::DoAction(const ServerCallContext& context, ARROW_ASSIGN_OR_RAISE(ActionCreatePreparedStatementRequest internal_command, ParseActionCreatePreparedStatementRequest(any_command)); - ARROW_ASSIGN_OR_RAISE(auto result, CreatePreparedStatement(internal_command, context)) + 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); @@ -431,14 +431,16 @@ Status FlightSqlServerBase::DoAction(const ServerCallContext& context, 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()); + 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}})); + *result_stream = std::unique_ptr( + new SimpleResultStream({Result{buf}})); return Status::OK(); } else if (action.type == FlightSqlServerBase::kClosePreparedStatementActionType.type) { @@ -450,7 +452,7 @@ Status FlightSqlServerBase::DoAction(const ServerCallContext& context, ARROW_ASSIGN_OR_RAISE(ActionClosePreparedStatementRequest internal_command, ParseActionClosePreparedStatementRequest(any)); - return ClosePreparedStatement(internal_command, context, result_stream); + return ClosePreparedStatement(context, internal_command, result_stream); } return Status::Invalid("The defined request is invalid."); } @@ -466,33 +468,33 @@ Status FlightSqlServerBase::DoGetCatalogs(const ServerCallContext& context, return Status::NotImplemented("DoGetCatalogs not implemented"); } -Status FlightSqlServerBase::GetFlightInfoStatement(const StatementQuery& command, - const ServerCallContext& context, +Status FlightSqlServerBase::GetFlightInfoStatement(const ServerCallContext& context, + const StatementQuery& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoStatement not implemented"); } -Status FlightSqlServerBase::DoGetStatement(const StatementQueryTicket& command, - const ServerCallContext& context, +Status FlightSqlServerBase::DoGetStatement(const ServerCallContext& context, + const StatementQueryTicket& command, std::unique_ptr* result) { return Status::NotImplemented("DoGetStatement not implemented"); } Status FlightSqlServerBase::GetFlightInfoPreparedStatement( - const PreparedStatementQuery& command, const ServerCallContext& context, + const ServerCallContext& context, const PreparedStatementQuery& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoPreparedStatement not implemented"); } Status FlightSqlServerBase::DoGetPreparedStatement( - const PreparedStatementQuery& command, const ServerCallContext& context, + const ServerCallContext& context, const PreparedStatementQuery& command, std::unique_ptr* result) { return Status::NotImplemented("DoGetPreparedStatement not implemented"); } -Status FlightSqlServerBase::GetFlightInfoSqlInfo(const GetSqlInfo& command, - const ServerCallContext& context, +Status FlightSqlServerBase::GetFlightInfoSqlInfo(const ServerCallContext& context, + const GetSqlInfo& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { assert(info != nullptr); @@ -514,8 +516,8 @@ void FlightSqlServerBase::RegisterSqlInfo(int32_t id, const SqlInfoResult& resul sql_info_id_to_result_[id] = result; } -Status FlightSqlServerBase::DoGetSqlInfo(const GetSqlInfo& command, - const ServerCallContext& context, +Status FlightSqlServerBase::DoGetSqlInfo(const ServerCallContext& context, + const GetSqlInfo& command, std::unique_ptr* result) { assert(result != nullptr); @@ -579,28 +581,28 @@ Status FlightSqlServerBase::DoGetSqlInfo(const GetSqlInfo& command, return Status::OK(); } -Status FlightSqlServerBase::GetFlightInfoSchemas(const GetSchemas& command, - const ServerCallContext& context, +Status FlightSqlServerBase::GetFlightInfoSchemas(const ServerCallContext& context, + const GetSchemas& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoSchemas not implemented"); } -Status FlightSqlServerBase::DoGetSchemas(const GetSchemas& command, - const ServerCallContext& context, +Status FlightSqlServerBase::DoGetSchemas(const ServerCallContext& context, + const GetSchemas& command, std::unique_ptr* result) { return Status::NotImplemented("DoGetSchemas not implemented"); } -Status FlightSqlServerBase::GetFlightInfoTables(const GetTables& command, - const ServerCallContext& context, +Status FlightSqlServerBase::GetFlightInfoTables(const ServerCallContext& context, + const GetTables& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoTables not implemented"); } -Status FlightSqlServerBase::DoGetTables(const GetTables& command, - const ServerCallContext& context, +Status FlightSqlServerBase::DoGetTables(const ServerCallContext& context, + const GetTables& command, std::unique_ptr* result) { return Status::NotImplemented("DoGetTables not implemented"); } @@ -616,84 +618,84 @@ Status FlightSqlServerBase::DoGetTableTypes(const ServerCallContext& context, return Status::NotImplemented("DoGetTableTypes not implemented"); } -Status FlightSqlServerBase::GetFlightInfoPrimaryKeys(const GetPrimaryKeys& command, - const ServerCallContext& context, +Status FlightSqlServerBase::GetFlightInfoPrimaryKeys(const ServerCallContext& context, + const GetPrimaryKeys& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoPrimaryKeys not implemented"); } -Status FlightSqlServerBase::DoGetPrimaryKeys(const GetPrimaryKeys& command, - const ServerCallContext& context, +Status FlightSqlServerBase::DoGetPrimaryKeys(const ServerCallContext& context, + const GetPrimaryKeys& command, std::unique_ptr* result) { return Status::NotImplemented("DoGetPrimaryKeys not implemented"); } -Status FlightSqlServerBase::GetFlightInfoExportedKeys(const GetExportedKeys& command, - const ServerCallContext& context, +Status FlightSqlServerBase::GetFlightInfoExportedKeys(const ServerCallContext& context, + const GetExportedKeys& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoExportedKeys not implemented"); } -Status FlightSqlServerBase::DoGetExportedKeys(const GetExportedKeys& command, - const ServerCallContext& context, +Status FlightSqlServerBase::DoGetExportedKeys(const ServerCallContext& context, + const GetExportedKeys& command, std::unique_ptr* result) { return Status::NotImplemented("DoGetExportedKeys not implemented"); } -Status FlightSqlServerBase::GetFlightInfoImportedKeys(const GetImportedKeys& command, - const ServerCallContext& context, +Status FlightSqlServerBase::GetFlightInfoImportedKeys(const ServerCallContext& context, + const GetImportedKeys& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoImportedKeys not implemented"); } -Status FlightSqlServerBase::DoGetImportedKeys(const GetImportedKeys& command, - const ServerCallContext& context, +Status FlightSqlServerBase::DoGetImportedKeys(const ServerCallContext& context, + const GetImportedKeys& command, std::unique_ptr* result) { return Status::NotImplemented("DoGetImportedKeys not implemented"); } Status FlightSqlServerBase::GetFlightInfoCrossReference( - const GetCrossReference& command, const ServerCallContext& context, + const ServerCallContext& context, const GetCrossReference& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { return Status::NotImplemented("GetFlightInfoCrossReference not implemented"); } Status FlightSqlServerBase::DoGetCrossReference( - const GetCrossReference& command, const ServerCallContext& context, + const ServerCallContext& context, const GetCrossReference& command, std::unique_ptr* result) { return Status::NotImplemented("DoGetCrossReference not implemented"); } arrow::Result FlightSqlServerBase::CreatePreparedStatement( - const ActionCreatePreparedStatementRequest& request, - const ServerCallContext& context) { + const ServerCallContext& context, + const ActionCreatePreparedStatementRequest& request) { return Status::NotImplemented("CreatePreparedStatement not implemented"); } Status FlightSqlServerBase::ClosePreparedStatement( - const ActionClosePreparedStatementRequest& request, const ServerCallContext& context, - std::unique_ptr* p_ptr) { + const ServerCallContext& context, const ActionClosePreparedStatementRequest& request, + std::unique_ptr* result) { return Status::NotImplemented("ClosePreparedStatement not implemented"); } Status FlightSqlServerBase::DoPutPreparedStatementQuery( - const PreparedStatementQuery& command, const ServerCallContext& context, + const ServerCallContext& context, const PreparedStatementQuery& command, FlightMessageReader* reader, FlightMetadataWriter* writer) { return Status::NotImplemented("DoPutPreparedStatementQuery not implemented"); } arrow::Result FlightSqlServerBase::DoPutPreparedStatementUpdate( - const PreparedStatementUpdate& command, const ServerCallContext& context, + const ServerCallContext& context, const PreparedStatementUpdate& command, FlightMessageReader* reader) { return Status::NotImplemented("DoPutPreparedStatementUpdate not implemented"); } arrow::Result FlightSqlServerBase::DoPutCommandStatementUpdate( - const StatementUpdate& command, const ServerCallContext& context, + const ServerCallContext& context, const StatementUpdate& command, std::unique_ptr& reader) { return Status::NotImplemented("DoPutCommandStatementUpdate not implemented"); } @@ -704,7 +706,8 @@ std::shared_ptr SqlSchema::GetCatalogsSchema() { std::shared_ptr SqlSchema::GetSchemasSchema() { return arrow::schema( - {field("catalog_name", utf8()), field("schema_name", utf8(), false)}); + {field("catalog_name", utf8()), + field("schema_name", utf8(), false)}); } std::shared_ptr SqlSchema::GetTablesSchema() { diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index c5497b1cee9..0dd28217b6a 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -161,47 +161,49 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { std::unique_ptr* result) override; /// \brief Gets a FlightInfo for executing a SQL query. - /// \param[in] command The StatementQuery object containing the SQL statement. /// \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. /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status GetFlightInfoStatement(const StatementQuery& command, - const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual Status + GetFlightInfoStatement(const ServerCallContext &context, const StatementQuery &command, + const FlightDescriptor &descriptor, + std::unique_ptr *info); /// \brief Gets a FlightDataStream containing the query results. - /// \param[in] command The StatementQueryTicket containing the statement handle. /// \param[in] context Per-call context. + /// \param[in] command The StatementQueryTicket containing the statement handle. + /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetStatement(const StatementQueryTicket& command, - const ServerCallContext& context, - std::unique_ptr* result); + virtual Status + DoGetStatement(const ServerCallContext &context, const StatementQueryTicket &command, + std::unique_ptr *result); /// \brief Gets 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] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoPreparedStatement(const PreparedStatementQuery& command, - const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual Status GetFlightInfoPreparedStatement(const ServerCallContext &context, + const PreparedStatementQuery &command, + const FlightDescriptor &descriptor, + std::unique_ptr *info); /// \brief Gets 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. - /// \param[in] context Per-call context. + /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetPreparedStatement(const PreparedStatementQuery& command, - const ServerCallContext& context, - std::unique_ptr* result); + virtual Status DoGetPreparedStatement(const ServerCallContext &context, + const PreparedStatementQuery &command, + std::unique_ptr *result); /// \brief Gets a FlightInfo for listing catalogs. /// \param[in] context Per-call context. @@ -220,67 +222,67 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { std::unique_ptr* result); /// \brief Gets 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] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status GetFlightInfoSqlInfo(const GetSqlInfo& command, - const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual Status + GetFlightInfoSqlInfo(const ServerCallContext &context, const GetSqlInfo &command, + const FlightDescriptor &descriptor, + std::unique_ptr *info); /// \brief Gets 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. - /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetSqlInfo(const GetSqlInfo& command, const ServerCallContext& context, - std::unique_ptr* result); + virtual Status DoGetSqlInfo(const ServerCallContext &context, const GetSqlInfo &command, + std::unique_ptr *result); /// \brief Gets a FlightInfo for listing schemas. + /// \param[in] context Per-call context. /// \param[in] command The GetSchemas object which may contain filters for /// catalog and schema name. - /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status GetFlightInfoSchemas(const GetSchemas& command, - const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual Status + GetFlightInfoSchemas(const ServerCallContext &context, const GetSchemas &command, + const FlightDescriptor &descriptor, + std::unique_ptr *info); /// \brief Gets a FlightDataStream containing the list of schemas. + /// \param[in] context Per-call context. /// \param[in] command The GetSchemas object which may contain filters for /// catalog and schema name. - /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetSchemas(const GetSchemas& command, const ServerCallContext& context, - std::unique_ptr* result); + virtual Status DoGetSchemas(const ServerCallContext &context, const GetSchemas &command, + std::unique_ptr *result); ///\brief Gets 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] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status GetFlightInfoTables(const GetTables& command, - const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual Status + GetFlightInfoTables(const ServerCallContext &context, const GetTables &command, + const FlightDescriptor &descriptor, + std::unique_ptr *info); /// \brief Gets 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. - /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetTables(const GetTables& command, const ServerCallContext& context, - std::unique_ptr* result); + virtual Status DoGetTables(const ServerCallContext &context, const GetTables &command, + std::unique_ptr *result); /// \brief Gets a FlightInfo to extract information about the table types. /// \param[in] context Per-call context. @@ -300,155 +302,155 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { std::unique_ptr* result); /// \brief Gets 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] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoPrimaryKeys(const GetPrimaryKeys& command, - const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual Status GetFlightInfoPrimaryKeys(const ServerCallContext &context, + const GetPrimaryKeys &command, + const FlightDescriptor &descriptor, + std::unique_ptr *info); /// \brief Gets 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. - /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetPrimaryKeys(const GetPrimaryKeys& command, - const ServerCallContext& context, - std::unique_ptr* result); + virtual Status + DoGetPrimaryKeys(const ServerCallContext &context, const GetPrimaryKeys &command, + std::unique_ptr *result); /// \brief Gets 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] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoExportedKeys(const GetExportedKeys& command, - const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual Status GetFlightInfoExportedKeys(const ServerCallContext &context, + const GetExportedKeys &command, + const FlightDescriptor &descriptor, + std::unique_ptr *info); /// \brief Gets 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. - /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetExportedKeys(const GetExportedKeys& command, - const ServerCallContext& context, - std::unique_ptr* result); + virtual Status + DoGetExportedKeys(const ServerCallContext &context, const GetExportedKeys &command, + std::unique_ptr *result); /// \brief Gets 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] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoImportedKeys(const GetImportedKeys& command, - const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual Status GetFlightInfoImportedKeys(const ServerCallContext &context, + const GetImportedKeys &command, + const FlightDescriptor &descriptor, + std::unique_ptr *info); /// \brief Gets 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. - /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetImportedKeys(const GetImportedKeys& command, - const ServerCallContext& context, - std::unique_ptr* result); + virtual Status + DoGetImportedKeys(const ServerCallContext &context, const GetImportedKeys &command, + std::unique_ptr *result); /// \brief Gets 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] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoCrossReference(const GetCrossReference& command, - const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual Status GetFlightInfoCrossReference(const ServerCallContext &context, + const GetCrossReference &command, + const FlightDescriptor &descriptor, + std::unique_ptr *info); /// \brief Gets 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. - /// \param[in] context Per-call context. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetCrossReference(const GetCrossReference& command, - const ServerCallContext& context, - std::unique_ptr* result); + virtual Status + DoGetCrossReference(const ServerCallContext &context, const GetCrossReference &command, + std::unique_ptr *result); /// \brief Executes an update SQL statement. - /// \param[in] command The StatementUpdate object containing the SQL statement. /// \param[in] context The call context. + /// \param[in] command The StatementUpdate object containing the SQL statement. /// \param[in] reader a sequence of uploaded record batches. /// \return The changed record count. - virtual arrow::Result DoPutCommandStatementUpdate( - const StatementUpdate& command, const ServerCallContext& context, - std::unique_ptr& reader); + virtual arrow::Result + DoPutCommandStatementUpdate(const ServerCallContext &context, + const StatementUpdate &command, + std::unique_ptr &reader); /// \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. - /// \param[in] context The call context. /// \return A ActionCreatePreparedStatementResult containing the dataset /// and parameter schemas and a handle for created statement. - virtual arrow::Result CreatePreparedStatement( - const ActionCreatePreparedStatementRequest& request, - const ServerCallContext& context); + virtual arrow::Result + CreatePreparedStatement(const ServerCallContext &context, + const ActionCreatePreparedStatementRequest &request); /// \brief Closes a prepared statement. + /// \param[in] context The call context. /// \param[in] request The ActionClosePreparedStatementRequest object containing the /// prepared statement handle. - /// \param[in] context The call context. /// \param[out] result Empty ResultStream. virtual Status ClosePreparedStatement( - const ActionClosePreparedStatementRequest& request, - const ServerCallContext& context, std::unique_ptr* result); + const ServerCallContext &context, + const ActionClosePreparedStatementRequest &request, + std::unique_ptr *result); /// \brief Binds parameters to given prepared statement. + /// \param[in] context The call context. /// \param[in] command The PreparedStatementQuery object containing the /// prepared statement handle. - /// \param[in] context The call context. /// \param[in] reader A sequence of uploaded record batches. /// \param[in] writer Send metadata back to the client. - virtual Status DoPutPreparedStatementQuery(const PreparedStatementQuery& command, - const ServerCallContext& context, - FlightMessageReader* reader, - FlightMetadataWriter* writer); + virtual Status DoPutPreparedStatementQuery(const ServerCallContext &context, + const PreparedStatementQuery &command, + FlightMessageReader *reader, + FlightMetadataWriter *writer); /// \brief Executes an update SQL prepared statement. + /// \param[in] context The call context. /// \param[in] command The PreparedStatementUpdate object containing the /// prepared statement handle. - /// \param[in] context The call context. /// \param[in] reader a sequence of uploaded record batches. /// \return The changed record count. - virtual arrow::Result DoPutPreparedStatementUpdate( - const PreparedStatementUpdate& command, const ServerCallContext& context, - FlightMessageReader* reader); - - protected: - static std::string CreateStatementQueryTicket(const std::string& statement_handle); + virtual arrow::Result + DoPutPreparedStatementUpdate(const ServerCallContext &context, + const PreparedStatementUpdate &command, + FlightMessageReader *reader); /// \brief Registers a new SqlInfo result. /// \param[in] id the SqlInfo identifier. From 6908d3851b1c85a9db15bb58c20ab818d7a98ef2 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 10 Nov 2021 17:21:02 -0300 Subject: [PATCH 209/237] Change status type on sqlite classes --- cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc | 4 ++-- cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc index e89c7ff31cf..56d76b6b998 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc @@ -178,7 +178,7 @@ arrow::Result> SQLiteFlightSqlServer::Cre err_msg += "Unable to start SQLite. Insufficient memory"; } - return Status::RError(err_msg); + return Status::Invalid(err_msg); } std::shared_ptr result(new SQLiteFlightSqlServer(db)); @@ -186,7 +186,7 @@ arrow::Result> SQLiteFlightSqlServer::Cre result->RegisterSqlInfo(id_to_result.first, id_to_result.second); } - ARROW_UNUSED(result->ExecuteSql(R"( + ARROW_RETURN_NOT_OK(result->ExecuteSql(R"( CREATE TABLE foreignTable ( id INTEGER PRIMARY KEY AUTOINCREMENT, foreignName varchar(100), diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc index f0dfc8da724..6f0d0bcc5c0 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc @@ -59,7 +59,7 @@ arrow::Result> SqliteStatement::Create( err_msg += std::string(sqlite3_errmsg(db)); } } - return Status::RError(err_msg); + return Status::Invalid(err_msg); } std::shared_ptr result(new SqliteStatement(db, stmt)); From 45b58cb00a48a751b231ecd054fe48e6e8741767 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Wed, 10 Nov 2021 17:31:05 -0300 Subject: [PATCH 210/237] Add documentation on public method GetFlightInfo --- cpp/src/arrow/flight/flight_sql/client.h | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/cpp/src/arrow/flight/flight_sql/client.h b/cpp/src/arrow/flight/flight_sql/client.h index c3c571d6b8b..ab023aac870 100644 --- a/cpp/src/arrow/flight/flight_sql/client.h +++ b/cpp/src/arrow/flight/flight_sql/client.h @@ -162,6 +162,12 @@ class ARROW_EXPORT FlightSqlClient { arrow::Result> Prepare( const FlightCallOptions& options, const std::string& query); + /// \brief Retrieve the flightInfo. It is public because it is been used by the + /// anonymous function GetFlightInfoForCommand. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] descriptor The flight descriptior. + /// \param[out] info The flight info with the metadata. + /// \return Status. virtual Status GetFlightInfo(const FlightCallOptions& options, const FlightDescriptor& descriptor, std::unique_ptr* info) { From c778682eee06d1dd6bb744ce979b165b8c1b8bc4 Mon Sep 17 00:00:00 2001 From: Jose Almeida Date: Thu, 11 Nov 2021 08:50:21 -0300 Subject: [PATCH 211/237] Change Status to arrow::Result on CreateStatementQueryTicket method --- .../arrow/flight/flight_sql/example/sqlite_server.cc | 2 +- cpp/src/arrow/flight/flight_sql/server.cc | 10 ++++++---- cpp/src/arrow/flight/flight_sql/server.h | 7 +++---- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc index 56d76b6b998..03e4a66d876 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc @@ -265,7 +265,7 @@ Status SQLiteFlightSqlServer::GetFlightInfoStatement(const ServerCallContext& co ARROW_RETURN_NOT_OK(statement->GetSchema(&schema)); std::string ticket_string; - ARROW_RETURN_NOT_OK(CreateStatementQueryTicket(query, &ticket_string)); + ARROW_ASSIGN_OR_RAISE(ticket_string, CreateStatementQueryTicket(query)); std::vector endpoints{FlightEndpoint{{ticket_string}, {}}}; ARROW_ASSIGN_OR_RAISE(auto result, FlightInfo::Make(*schema, descriptor, endpoints, -1, -1)) diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index 85504b79f1d..9c23041e03c 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -40,18 +40,20 @@ namespace sql { namespace pb = arrow::flight::protocol; -Status CreateStatementQueryTicket(const std::string& statement_handle, - std::string* ticket_string) { +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); - if (!ticket.SerializeToString(ticket_string)) { + std::string ticket_string; + + if (!ticket.SerializeToString(&ticket_string)) { return Status::IOError("Invalid ticket."); } - return Status::OK(); + return ticket_string; } arrow::Result ParseCommandGetCrossReference( diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index 0dd28217b6a..696fc60b666 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -123,10 +123,9 @@ struct ActionCreatePreparedStatementResult { /// \brief A free function responsible to create a ticket for the statement operation. /// It is used by the server implementation. /// \param[in] statement_handle The statement handle that will originate the ticket. -/// \param[out] ticket_string The parsed ticket as a string. -/// \return Status -Status CreateStatementQueryTicket(const std::string &statement_handle, - std::string *ticket_string); +/// \return The parsed ticket as an string. +arrow::Result CreateStatementQueryTicket( + const std::string &statement_handle); class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { private: From 2e48187c2c683ea82f79d1abef520b385ed66b69 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Thu, 11 Nov 2021 13:53:06 -0300 Subject: [PATCH 212/237] Fix method docs on server.h --- cpp/src/arrow/flight/flight_sql/server.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index 696fc60b666..913becd2cab 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -173,7 +173,6 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \brief Gets a FlightDataStream containing the query results. /// \param[in] context Per-call context. /// \param[in] command The StatementQueryTicket containing the statement handle. - /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] result The FlightDataStream containing the results. /// \return Status. virtual Status @@ -197,7 +196,6 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \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. /// \param[out] result The FlightDataStream containing the results. /// \return Status. virtual Status DoGetPreparedStatement(const ServerCallContext &context, From ec1c4d0e2b36ef51a18cd28fdad6783bd3d69430 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 11 Nov 2021 13:21:25 -0300 Subject: [PATCH 213/237] Fix ODR violation when linking protobuf --- cpp/src/arrow/flight/CMakeLists.txt | 18 +++++++++++++++++- cpp/src/arrow/flight/flight_sql/CMakeLists.txt | 12 +++++------- 2 files changed, 22 insertions(+), 8 deletions(-) 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/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index 61d2814a9b6..62277df08a8 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -36,7 +36,8 @@ set_source_files_properties(${FLIGHT_SQL_GENERATED_PROTO_FILES} PROPERTIES GENER add_custom_target(flight_sql_protobuf_gen ALL DEPENDS ${FLIGHT_SQL_GENERATED_PROTO_FILES}) -set(ARROW_FLIGHT_SQL_SRCS server.cc sql_info_util.cc client.cc) +set(ARROW_FLIGHT_SQL_SRCS server.cc sql_info_util.cc client.cc + "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc") add_arrow_lib(arrow_flight_sql CMAKE_PACKAGE_NAME @@ -58,7 +59,7 @@ add_arrow_lib(arrow_flight_sql STATIC_LINK_LIBS arrow_flight_static) -if(ARROW_TEST_LINKAGE STREQUAL "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}) @@ -67,8 +68,7 @@ else() ${ARROW_TEST_LINK_LIBS}) endif() -add_executable(flight_sql_test_app test_app_cli.cc - ${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc) +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}) @@ -76,7 +76,6 @@ add_arrow_test(flight_sql_test SOURCES client_test.cc server_test.cc - "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc" STATIC_LINK_LIBS ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} LABELS @@ -93,8 +92,7 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) example/sqlite_statement.cc example/sqlite_statement_batch_reader.cc example/sqlite_server.cc - example/sqlite_tables_schema_batch_reader.cc - "${CMAKE_CURRENT_BINARY_DIR}/FlightSql.pb.cc") + example/sqlite_tables_schema_batch_reader.cc) target_link_libraries(flight_sql_test_server PRIVATE ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} ${GFLAGS_LIBRARIES} ${SQLite3_LIBRARIES}) From 3cee40c5ddf7fc066e2d473202576284a946e66b Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 11 Nov 2021 14:56:58 -0300 Subject: [PATCH 214/237] Fix code style issues --- cpp/src/arrow/flight/flight_sql/server.cc | 11 +- cpp/src/arrow/flight/flight_sql/server.h | 156 +++++++++++----------- 2 files changed, 81 insertions(+), 86 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index 9c23041e03c..6cefcb036a7 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -41,7 +41,7 @@ namespace sql { namespace pb = arrow::flight::protocol; arrow::Result CreateStatementQueryTicket( - const std::string& statement_handle) { + const std::string& statement_handle) { protocol::sql::TicketStatementQuery ticket_statement_query; ticket_statement_query.set_statement_handle(statement_handle); @@ -433,16 +433,14 @@ Status FlightSqlServerBase::DoAction(const ServerCallContext& context, 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()); + 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}})); + *result_stream = std::unique_ptr(new SimpleResultStream({Result{buf}})); return Status::OK(); } else if (action.type == FlightSqlServerBase::kClosePreparedStatementActionType.type) { @@ -708,8 +706,7 @@ std::shared_ptr SqlSchema::GetCatalogsSchema() { std::shared_ptr SqlSchema::GetSchemasSchema() { return arrow::schema( - {field("catalog_name", utf8()), - field("schema_name", utf8(), false)}); + {field("catalog_name", utf8()), field("schema_name", utf8(), false)}); } std::shared_ptr SqlSchema::GetTablesSchema() { diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index 913becd2cab..854c4a25650 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -125,7 +125,7 @@ struct ActionCreatePreparedStatementResult { /// \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); + const std::string& statement_handle); class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { private: @@ -165,19 +165,19 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status - GetFlightInfoStatement(const ServerCallContext &context, const StatementQuery &command, - const FlightDescriptor &descriptor, - std::unique_ptr *info); + virtual Status GetFlightInfoStatement(const ServerCallContext& context, + const StatementQuery& command, + const FlightDescriptor& descriptor, + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the query results. /// \param[in] context Per-call context. /// \param[in] command The StatementQueryTicket containing the statement handle. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status - DoGetStatement(const ServerCallContext &context, const StatementQueryTicket &command, - std::unique_ptr *result); + virtual Status DoGetStatement(const ServerCallContext& context, + const StatementQueryTicket& command, + std::unique_ptr* result); /// \brief Gets a FlightInfo for executing an already created prepared statement. /// \param[in] context Per-call context. @@ -187,10 +187,10 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoPreparedStatement(const ServerCallContext &context, - const PreparedStatementQuery &command, - const FlightDescriptor &descriptor, - std::unique_ptr *info); + virtual Status GetFlightInfoPreparedStatement(const ServerCallContext& context, + const PreparedStatementQuery& command, + const FlightDescriptor& descriptor, + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the prepared statement query results. /// \param[in] context Per-call context. @@ -198,9 +198,9 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// prepared statement handle. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetPreparedStatement(const ServerCallContext &context, - const PreparedStatementQuery &command, - std::unique_ptr *result); + virtual Status DoGetPreparedStatement(const ServerCallContext& context, + const PreparedStatementQuery& command, + std::unique_ptr* result); /// \brief Gets a FlightInfo for listing catalogs. /// \param[in] context Per-call context. @@ -225,10 +225,10 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status - GetFlightInfoSqlInfo(const ServerCallContext &context, const GetSqlInfo &command, - const FlightDescriptor &descriptor, - std::unique_ptr *info); + virtual Status GetFlightInfoSqlInfo(const ServerCallContext& context, + const GetSqlInfo& command, + const FlightDescriptor& descriptor, + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the list of SqlInfo results. /// \param[in] context Per-call context. @@ -236,8 +236,8 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// to be returned. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetSqlInfo(const ServerCallContext &context, const GetSqlInfo &command, - std::unique_ptr *result); + virtual Status DoGetSqlInfo(const ServerCallContext& context, const GetSqlInfo& command, + std::unique_ptr* result); /// \brief Gets a FlightInfo for listing schemas. /// \param[in] context Per-call context. @@ -246,10 +246,10 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status - GetFlightInfoSchemas(const ServerCallContext &context, const GetSchemas &command, - const FlightDescriptor &descriptor, - std::unique_ptr *info); + virtual Status GetFlightInfoSchemas(const ServerCallContext& context, + const GetSchemas& command, + const FlightDescriptor& descriptor, + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the list of schemas. /// \param[in] context Per-call context. @@ -257,8 +257,8 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// catalog and schema name. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetSchemas(const ServerCallContext &context, const GetSchemas &command, - std::unique_ptr *result); + virtual Status DoGetSchemas(const ServerCallContext& context, const GetSchemas& command, + std::unique_ptr* result); ///\brief Gets a FlightInfo for listing tables. /// \param[in] context Per-call context. @@ -267,10 +267,10 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \param[in] descriptor The descriptor identifying the data stream. /// \param[out] info The FlightInfo describing where to access the dataset. /// \return Status. - virtual Status - GetFlightInfoTables(const ServerCallContext &context, const GetTables &command, - const FlightDescriptor &descriptor, - std::unique_ptr *info); + virtual Status GetFlightInfoTables(const ServerCallContext& context, + const GetTables& command, + const FlightDescriptor& descriptor, + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the list of tables. /// \param[in] context Per-call context. @@ -278,8 +278,8 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// catalog, schema and table names. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status DoGetTables(const ServerCallContext &context, const GetTables &command, - std::unique_ptr *result); + virtual Status DoGetTables(const ServerCallContext& context, const GetTables& command, + std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about the table types. /// \param[in] context Per-call context. @@ -306,10 +306,10 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoPrimaryKeys(const ServerCallContext &context, - const GetPrimaryKeys &command, - const FlightDescriptor &descriptor, - std::unique_ptr *info); + virtual Status GetFlightInfoPrimaryKeys(const ServerCallContext& context, + const GetPrimaryKeys& command, + const FlightDescriptor& descriptor, + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the data related to the primary and /// foreign @@ -319,9 +319,9 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// to execute the request. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status - DoGetPrimaryKeys(const ServerCallContext &context, const GetPrimaryKeys &command, - std::unique_ptr *result); + virtual Status DoGetPrimaryKeys(const ServerCallContext& context, + const GetPrimaryKeys& command, + std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about foreign and primary keys. /// \param[in] context Per-call context. @@ -331,10 +331,10 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoExportedKeys(const ServerCallContext &context, - const GetExportedKeys &command, - const FlightDescriptor &descriptor, - std::unique_ptr *info); + virtual Status GetFlightInfoExportedKeys(const ServerCallContext& context, + const GetExportedKeys& command, + const FlightDescriptor& descriptor, + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the data related to the foreign and /// primary @@ -344,9 +344,9 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// to execute the request. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status - DoGetExportedKeys(const ServerCallContext &context, const GetExportedKeys &command, - std::unique_ptr *result); + virtual Status DoGetExportedKeys(const ServerCallContext& context, + const GetExportedKeys& command, + std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about foreign and primary keys. /// \param[in] context Per-call context. @@ -356,10 +356,10 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoImportedKeys(const ServerCallContext &context, - const GetImportedKeys &command, - const FlightDescriptor &descriptor, - std::unique_ptr *info); + virtual Status GetFlightInfoImportedKeys(const ServerCallContext& context, + const GetImportedKeys& command, + const FlightDescriptor& descriptor, + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the data related to the foreign and /// primary keys. @@ -368,9 +368,9 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// to execute the request. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status - DoGetImportedKeys(const ServerCallContext &context, const GetImportedKeys &command, - std::unique_ptr *result); + virtual Status DoGetImportedKeys(const ServerCallContext& context, + const GetImportedKeys& command, + std::unique_ptr* result); /// \brief Gets a FlightInfo to extract information about foreign and primary keys. /// \param[in] context Per-call context. @@ -381,10 +381,10 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \param[out] info The FlightInfo describing where to access the /// dataset. /// \return Status. - virtual Status GetFlightInfoCrossReference(const ServerCallContext &context, - const GetCrossReference &command, - const FlightDescriptor &descriptor, - std::unique_ptr *info); + virtual Status GetFlightInfoCrossReference(const ServerCallContext& context, + const GetCrossReference& command, + const FlightDescriptor& descriptor, + std::unique_ptr* info); /// \brief Gets a FlightDataStream containing the data related to the foreign and /// primary keys. @@ -393,19 +393,18 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// to execute the request. /// \param[out] result The FlightDataStream containing the results. /// \return Status. - virtual Status - DoGetCrossReference(const ServerCallContext &context, const GetCrossReference &command, - std::unique_ptr *result); + virtual Status DoGetCrossReference(const ServerCallContext& context, + const GetCrossReference& command, + std::unique_ptr* result); /// \brief Executes an update SQL statement. /// \param[in] context The call context. /// \param[in] command The StatementUpdate object containing the SQL statement. /// \param[in] reader a sequence of uploaded record batches. /// \return The changed record count. - virtual arrow::Result - DoPutCommandStatementUpdate(const ServerCallContext &context, - const StatementUpdate &command, - std::unique_ptr &reader); + virtual arrow::Result DoPutCommandStatementUpdate( + const ServerCallContext& context, const StatementUpdate& command, + std::unique_ptr& reader); /// \brief Create a prepared statement from given SQL statement. /// \param[in] context The call context. @@ -413,9 +412,9 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// 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); + virtual arrow::Result CreatePreparedStatement( + const ServerCallContext& context, + const ActionCreatePreparedStatementRequest& request); /// \brief Closes a prepared statement. /// \param[in] context The call context. @@ -423,9 +422,9 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// prepared statement handle. /// \param[out] result Empty ResultStream. virtual Status ClosePreparedStatement( - const ServerCallContext &context, - const ActionClosePreparedStatementRequest &request, - std::unique_ptr *result); + const ServerCallContext& context, + const ActionClosePreparedStatementRequest& request, + std::unique_ptr* result); /// \brief Binds parameters to given prepared statement. /// \param[in] context The call context. @@ -433,10 +432,10 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// 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); + virtual Status DoPutPreparedStatementQuery(const ServerCallContext& context, + const PreparedStatementQuery& command, + FlightMessageReader* reader, + FlightMetadataWriter* writer); /// \brief Executes an update SQL prepared statement. /// \param[in] context The call context. @@ -444,10 +443,9 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// 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); + virtual arrow::Result DoPutPreparedStatementUpdate( + const ServerCallContext& context, const PreparedStatementUpdate& command, + FlightMessageReader* reader); /// \brief Registers a new SqlInfo result. /// \param[in] id the SqlInfo identifier. From 6c98d52656971a00cd863c40ca87708a6c1e4488 Mon Sep 17 00:00:00 2001 From: Juscelino Junior Date: Thu, 11 Nov 2021 16:38:45 -0300 Subject: [PATCH 215/237] Fix FindSQLite3Alt.cmake --- cpp/cmake_modules/FindSQLite3Alt.cmake | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/cpp/cmake_modules/FindSQLite3Alt.cmake b/cpp/cmake_modules/FindSQLite3Alt.cmake index 4e4704a2db1..73a45f098c6 100644 --- a/cpp/cmake_modules/FindSQLite3Alt.cmake +++ b/cpp/cmake_modules/FindSQLite3Alt.cmake @@ -32,10 +32,12 @@ 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 FAULT_MSG SQLite3_LIBRARY - SQLite3_INCLUDE_DIR) +find_package_handle_standard_args(SQLite3Alt REQUIRED_VARS SQLite3_LIBRARY + SQLite3_INCLUDE_DIR) mark_as_advanced(SQLite3_LIBRARY SQLite3_INCLUDE_DIR) -set(SQLite3_INCLUDE_DIRS ${SQLite3_INCLUDE_DIR}) -set(SQLite3_LIBRARIES ${SQLite3_LIBRARY}) +if(SQLite3Alt_FOUND) + set(SQLite3_INCLUDE_DIRS ${SQLite3_INCLUDE_DIR}) + set(SQLite3_LIBRARIES ${SQLite3_LIBRARY}) +endif() From f0555708f3b386382b1f32e28c825dce9802171b Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 12 Nov 2021 13:03:49 -0300 Subject: [PATCH 216/237] Enable Flight SQL C++ on CI --- ci/docker/conda-cpp.dockerfile | 1 + ci/scripts/cpp_build.sh | 3 ++- docker-compose.yml | 3 +++ 3 files changed, 6 insertions(+), 1 deletion(-) 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..cebf73f85e8 100755 --- a/ci/scripts/cpp_build.sh +++ b/ci/scripts/cpp_build.sh @@ -70,7 +70,8 @@ cmake \ -DARROW_EXTRA_ERROR_CONTEXT=${ARROW_EXTRA_ERROR_CONTEXT:-OFF} \ -DARROW_FILESYSTEM=${ARROW_FILESYSTEM:-ON} \ -DARROW_FLIGHT=${ARROW_FLIGHT:-OFF} \ - -DARROW_FUZZING=${ARROW_FUZZING:-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:-} \ -DARROW_GANDIVA=${ARROW_GANDIVA:-OFF} \ 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" From fd9bd948002ec393b05a1dfe50f4c8dceb5e4635 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 15 Nov 2021 11:25:53 -0300 Subject: [PATCH 217/237] Fix build when BUILD_EXAMPLES or BUILD_TESTS is OFF --- cpp/src/arrow/flight/flight_sql/CMakeLists.txt | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index 62277df08a8..d46454aa5ea 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -68,10 +68,6 @@ else() ${ARROW_TEST_LINK_LIBS}) endif() -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}) - add_arrow_test(flight_sql_test SOURCES client_test.cc @@ -97,6 +93,10 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) 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}) + if(ARROW_BUILD_TESTS) add_dependencies(arrow-flight-sql-test flight_sql_test_server) endif() From 76d04ea0dff846a1a3f5bf0176af1e287f4c05d3 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 15 Nov 2021 17:36:24 -0300 Subject: [PATCH 218/237] [C++] Fix comments on ratification PR (round 3) (#205) * Fix comments pointed on review * Replace boost::variant to arrow::util::variant * Remove unused macros and redundant definitions * Refactor sql_info_internal to prepopulate builder pointers on constructor * Replace ArrayFromJSON usage to ArrayFromVector for consistency on tests * Remove mention to GetFlightInfoForCommand from doxygen * Remove copy constructors on SQLInfo related visitors * Remove move constructors on SQLInfo related visitors --- .../arrow/flight/flight_sql/CMakeLists.txt | 4 +- cpp/src/arrow/flight/flight_sql/client.cc | 2 - cpp/src/arrow/flight/flight_sql/client.h | 7 +- .../flight_sql/example/sqlite_server.cc | 4 +- .../flight/flight_sql/example/sqlite_server.h | 370 +++++++++--------- .../flight_sql/example/sqlite_statement.cc | 6 +- .../example/sqlite_statement_batch_reader.cc | 2 +- cpp/src/arrow/flight/flight_sql/server.cc | 74 ++-- cpp/src/arrow/flight/flight_sql/server.h | 17 +- .../arrow/flight/flight_sql/server_test.cc | 31 +- .../flight/flight_sql/sql_info_internal.cc | 101 +++++ .../{sql_info_util.h => sql_info_internal.h} | 36 +- .../arrow/flight/flight_sql/sql_info_util.cc | 113 ------ .../arrow/flight/flight_sql/test_app_cli.cc | 2 +- 14 files changed, 377 insertions(+), 392 deletions(-) create mode 100644 cpp/src/arrow/flight/flight_sql/sql_info_internal.cc rename cpp/src/arrow/flight/flight_sql/{sql_info_util.h => sql_info_internal.h} (68%) delete mode 100644 cpp/src/arrow/flight/flight_sql/sql_info_util.cc diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index d46454aa5ea..f17fccfc342 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -36,7 +36,7 @@ set_source_files_properties(${FLIGHT_SQL_GENERATED_PROTO_FILES} PROPERTIES GENER add_custom_target(flight_sql_protobuf_gen ALL DEPENDS ${FLIGHT_SQL_GENERATED_PROTO_FILES}) -set(ARROW_FLIGHT_SQL_SRCS server.cc sql_info_util.cc client.cc +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 @@ -83,7 +83,7 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) add_executable(flight_sql_test_server server.cc - sql_info_util.cc + sql_info_internal.cc test_server_cli.cc example/sqlite_statement.cc example/sqlite_statement_batch_reader.cc diff --git a/cpp/src/arrow/flight/flight_sql/client.cc b/cpp/src/arrow/flight/flight_sql/client.cc index 82fc73104b0..e565c872cee 100644 --- a/cpp/src/arrow/flight/flight_sql/client.cc +++ b/cpp/src/arrow/flight/flight_sql/client.cc @@ -48,8 +48,6 @@ PreparedStatement::PreparedStatement(FlightSqlClient* client, std::string handle parameter_schema_(std::move(parameter_schema)), is_closed_(false) {} -FlightSqlClient::~FlightSqlClient() = default; - PreparedStatement::~PreparedStatement() { if (IsClosed()) return; diff --git a/cpp/src/arrow/flight/flight_sql/client.h b/cpp/src/arrow/flight/flight_sql/client.h index ab023aac870..f53960d8d6b 100644 --- a/cpp/src/arrow/flight/flight_sql/client.h +++ b/cpp/src/arrow/flight/flight_sql/client.h @@ -41,7 +41,7 @@ class ARROW_EXPORT FlightSqlClient { public: explicit FlightSqlClient(std::shared_ptr client); - ~FlightSqlClient(); + virtual ~FlightSqlClient() = default; /// \brief Execute a query on the server. /// \param[in] options RPC-layer hints for this call. @@ -162,12 +162,13 @@ class ARROW_EXPORT FlightSqlClient { arrow::Result> Prepare( const FlightCallOptions& options, const std::string& query); - /// \brief Retrieve the flightInfo. It is public because it is been used by the - /// anonymous function GetFlightInfoForCommand. + /// \brief Retrieve the FlightInfo. /// \param[in] options RPC-layer hints for this call. /// \param[in] descriptor The flight descriptior. /// \param[out] info The flight info with the metadata. /// \return Status. + // NOTE: This is public because it is been used by the anonymous + // function GetFlightInfoForCommand. virtual Status GetFlightInfo(const FlightCallOptions& options, const FlightDescriptor& descriptor, std::unique_ptr* info) { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc index 03e4a66d876..6d150bd814b 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc @@ -182,7 +182,7 @@ arrow::Result> SQLiteFlightSqlServer::Cre } std::shared_ptr result(new SQLiteFlightSqlServer(db)); - for (const auto& id_to_result : GetSqlInfoIdToResult()) { + for (const auto& id_to_result : GetSqlInfoResultMap()) { result->RegisterSqlInfo(id_to_result.first, id_to_result.second); } @@ -220,7 +220,7 @@ Status SQLiteFlightSqlServer::ExecuteSql(const std::string& sql) { error_msg = err_msg; } sqlite3_free(err_msg); - return Status::RError(error_msg); + return Status::ExecutionError(error_msg); } return Status::OK(); } diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h index 710316e778f..ec126cf4296 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h @@ -40,188 +40,194 @@ namespace flight_sql_pb = arrow::flight::protocol::sql; /// \brief Gets the mapping from SQL info ids to SqlInfoResult instances. /// \return the cache. -inline sql_info_id_to_result_t GetSqlInfoIdToResult() { - return {{flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_NAME, std::string("db_name")}, - {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_VERSION, std::string("sqlite 3")}, - {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_ARROW_VERSION, - std::string("7.0.0-SNAPSHOT" /* Only an example */)}, - {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_READ_ONLY, false}, - {flight_sql_pb::SqlInfo::SQL_DDL_CATALOG, - false /* SQLite 3 does not support catalogs */}, - {flight_sql_pb::SqlInfo::SQL_DDL_SCHEMA, - false /* SQLite 3 does not support schemas */}, - {flight_sql_pb::SqlInfo::SQL_DDL_TABLE, true}, - {flight_sql_pb::SqlInfo::SQL_IDENTIFIER_CASE, - int64_t(flight_sql_pb::SqlSupportedCaseSensitivity:: - SQL_CASE_SENSITIVITY_CASE_INSENSITIVE)}, - {flight_sql_pb::SqlInfo::SQL_IDENTIFIER_QUOTE_CHAR, std::string("\"")}, - {flight_sql_pb::SqlInfo::SQL_QUOTED_IDENTIFIER_CASE, - int64_t(flight_sql_pb::SqlSupportedCaseSensitivity:: - SQL_CASE_SENSITIVITY_CASE_INSENSITIVE)}, - {flight_sql_pb::SqlInfo::SQL_ALL_TABLES_ARE_SELECTABLE, true}, - {flight_sql_pb::SqlInfo::SQL_NULL_ORDERING, - int64_t(flight_sql_pb::SqlNullOrdering::SQL_NULLS_SORTED_AT_START)}, - {flight_sql_pb::SqlInfo::SQL_KEYWORDS, string_list_t({"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"})}, - {flight_sql_pb::SqlInfo::SQL_NUMERIC_FUNCTIONS, - string_list_t({"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"})}, - {flight_sql_pb::SqlInfo::SQL_STRING_FUNCTIONS, - string_list_t({"SUBSTR", "TRIM", "LTRIM", "RTRIM", "LENGTH", "REPLACE", - "UPPER", "LOWER", "INSTR"})}, - {flight_sql_pb::SqlInfo::SQL_SUPPORTS_CONVERT, - int32_to_int32_list_t( - {{flight_sql_pb::SqlSupportsConvert::SQL_CONVERT_BIGINT, - std::vector( - {flight_sql_pb::SqlSupportsConvert::SQL_CONVERT_INTEGER})}})}}; +inline SqlInfoResultMap GetSqlInfoResultMap() { + return { + {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_NAME, + SqlInfoResult(std::string("db_name"))}, + {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_VERSION, + SqlInfoResult(std::string("sqlite 3"))}, + {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_ARROW_VERSION, + SqlInfoResult(std::string("7.0.0-SNAPSHOT" /* Only an example */))}, + {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_READ_ONLY, SqlInfoResult(false)}, + {flight_sql_pb::SqlInfo::SQL_DDL_CATALOG, + SqlInfoResult(false /* SQLite 3 does not support catalogs */)}, + {flight_sql_pb::SqlInfo::SQL_DDL_SCHEMA, + SqlInfoResult(false /* SQLite 3 does not support schemas */)}, + {flight_sql_pb::SqlInfo::SQL_DDL_TABLE, SqlInfoResult(true)}, + {flight_sql_pb::SqlInfo::SQL_IDENTIFIER_CASE, + SqlInfoResult(int64_t(flight_sql_pb::SqlSupportedCaseSensitivity:: + SQL_CASE_SENSITIVITY_CASE_INSENSITIVE))}, + {flight_sql_pb::SqlInfo::SQL_IDENTIFIER_QUOTE_CHAR, + SqlInfoResult(std::string("\""))}, + {flight_sql_pb::SqlInfo::SQL_QUOTED_IDENTIFIER_CASE, + SqlInfoResult(int64_t(flight_sql_pb::SqlSupportedCaseSensitivity:: + SQL_CASE_SENSITIVITY_CASE_INSENSITIVE))}, + {flight_sql_pb::SqlInfo::SQL_ALL_TABLES_ARE_SELECTABLE, SqlInfoResult(true)}, + {flight_sql_pb::SqlInfo::SQL_NULL_ORDERING, + SqlInfoResult(int64_t(flight_sql_pb::SqlNullOrdering::SQL_NULLS_SORTED_AT_START))}, + {flight_sql_pb::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"}))}, + {flight_sql_pb::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"}))}, + {flight_sql_pb::SqlInfo::SQL_STRING_FUNCTIONS, + SqlInfoResult( + std::vector({"SUBSTR", "TRIM", "LTRIM", "RTRIM", "LENGTH", + "REPLACE", "UPPER", "LOWER", "INSTR"}))}, + {flight_sql_pb::SqlInfo::SQL_SUPPORTS_CONVERT, + SqlInfoResult(std::unordered_map>( + {{flight_sql_pb::SqlSupportsConvert::SQL_CONVERT_BIGINT, + std::vector( + {flight_sql_pb::SqlSupportsConvert::SQL_CONVERT_INTEGER})}}))}}; } /// \brief Convert a column type to a ArrowType. diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc index 6f0d0bcc5c0..c08323bb21a 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc @@ -108,7 +108,8 @@ SqliteStatement::~SqliteStatement() { sqlite3_finalize(stmt_); } Status SqliteStatement::Step(int* rc) { *rc = sqlite3_step(stmt_); if (*rc == SQLITE_ERROR) { - return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db_)); + return Status::ExecutionError("A SQLite runtime error has occurred: ", + sqlite3_errmsg(db_)); } return Status::OK(); @@ -117,7 +118,8 @@ Status SqliteStatement::Step(int* rc) { Status SqliteStatement::Reset(int* rc) { *rc = sqlite3_reset(stmt_); if (*rc == SQLITE_ERROR) { - return Status::RError("A SQLite runtime error has occurred: ", sqlite3_errmsg(db_)); + return Status::ExecutionError("A SQLite runtime error has occurred: ", + sqlite3_errmsg(db_)); } return Status::OK(); diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc index 149799a7ea0..4071c729a8a 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc @@ -19,7 +19,7 @@ #include -#include "arrow/api.h" +#include "arrow/builder.h" #include "arrow/flight/flight_sql/example/sqlite_statement.h" #define STRING_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index 6cefcb036a7..30eb5ac9d92 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -25,15 +25,12 @@ #include "arrow/api.h" #include "arrow/buffer.h" #include "arrow/flight/flight_sql/FlightSql.pb.h" +#include "arrow/type.h" +#include "arrow/util/logging.h" #define PROPERTY_TO_OPTIONAL(COMMAND, PROPERTY) \ COMMAND.has_##PROPERTY() ? util::make_optional(COMMAND.PROPERTY()) : util::nullopt; -#define REGISTER_FIELD(name, type, nullable) \ - const auto& name = field(ARROW_STRINGIFY(name), type, nullable); - -#define REGISTER_FIELD_NOT_NULL(name, type) REGISTER_FIELD(name, type, false) - namespace arrow { namespace flight { namespace sql { @@ -116,8 +113,7 @@ arrow::Result ParseCommandGetPrimaryKeys( } arrow::Result ParseCommandGetSqlInfo( - const google::protobuf::Any& any, - const sql_info_id_to_result_t& sql_info_id_to_result) { + 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."); @@ -519,38 +515,17 @@ void FlightSqlServerBase::RegisterSqlInfo(int32_t id, const SqlInfoResult& resul Status FlightSqlServerBase::DoGetSqlInfo(const ServerCallContext& context, const GetSqlInfo& command, std::unique_ptr* result) { - assert(result != nullptr); + DCHECK(result); MemoryPool* memory_pool = default_memory_pool(); UInt32Builder name_field_builder(memory_pool); - const std::shared_ptr string_value_field_builder( - new StringBuilder(memory_pool)); - const std::shared_ptr bool_value_field_builder( - new BooleanBuilder(memory_pool)); - const std::shared_ptr bigint_value_field_builder( - new Int64Builder(memory_pool)); - const std::shared_ptr int32_bitmask_field_builder( - new Int32Builder(memory_pool)); - const std::shared_ptr string_list_inner_string_field_builder( - new StringBuilder(memory_pool)); - const std::shared_ptr string_list_field_builder( - new ListBuilder(memory_pool, string_list_inner_string_field_builder)); - const std::shared_ptr int32_to_int32_list_key_builder( - new Int32Builder(memory_pool)); - const std::shared_ptr int32_to_int32_list_elements_in_value_list( - new Int32Builder(memory_pool)); - const std::shared_ptr int32_to_int32_list_value_builder( - new ListBuilder(memory_pool, int32_to_int32_list_elements_in_value_list)); - const std::shared_ptr int32_to_int32_list_map_field_builder(new MapBuilder( - memory_pool, int32_to_int32_list_key_builder, int32_to_int32_list_value_builder)); - std::vector> value_field_builders{ - string_value_field_builder, bool_value_field_builder, - bigint_value_field_builder, int32_bitmask_field_builder, - string_list_field_builder, int32_to_int32_list_map_field_builder}; - const std::shared_ptr& schema = SqlSchema::GetSqlInfoSchema(); - const auto& union_type = schema->fields()[1]->type(); // expected union type. - DenseUnionBuilder value_field_builder(memory_pool, value_field_builders, union_type); - internal::SqlInfoResultAppender sql_info_result_appender(value_field_builder); + std::unique_ptr value_field_builder; + const auto& value_field_type = std::static_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( + reinterpret_cast(*value_field_builder)); // Populate both name_field_builder and value_field_builder for each element // on command.info. @@ -565,17 +540,17 @@ Status FlightSqlServerBase::DoGetSqlInfo(const ServerCallContext& context, } ARROW_RETURN_NOT_OK(name_field_builder.Append(info)); ARROW_RETURN_NOT_OK( - boost::apply_visitor(sql_info_result_appender, sql_info_id_to_result_.at(info))); + arrow::util::visit(sql_info_result_appender, sql_info_id_to_result_.at(info))); } 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)); + ARROW_RETURN_NOT_OK(value_field_builder->Finish(&value)); auto row_count = static_cast(command.info.size()); const std::shared_ptr& batch = - RecordBatch::Make(schema, row_count, {name, value}); + RecordBatch::Make(SqlSchema::GetSqlInfoSchema(), row_count, {name, value}); ARROW_ASSIGN_OR_RAISE(const auto reader, RecordBatchReader::Make({batch})); *result = std::unique_ptr(new RecordBatchStream(reader)); return Status::OK(); @@ -754,17 +729,16 @@ std::shared_ptr SqlSchema::GetCrossReferenceSchema() { } std::shared_ptr SqlSchema::GetSqlInfoSchema() { - REGISTER_FIELD_NOT_NULL(name, uint32()) - REGISTER_FIELD_NOT_NULL(string_value, utf8()) - REGISTER_FIELD_NOT_NULL(bool_value, boolean()) - REGISTER_FIELD_NOT_NULL(bigint_value, int64()) - REGISTER_FIELD_NOT_NULL(int32_bitmask, int32()) - REGISTER_FIELD_NOT_NULL(string_list, list(utf8())) - REGISTER_FIELD_NOT_NULL(int32_to_int32_list_map, map(int32(), list(int32()))) - const FieldVector value_fields{string_value, bool_value, bigint_value, - int32_bitmask, string_list, int32_to_int32_list_map}; - REGISTER_FIELD_NOT_NULL(value, dense_union(value_fields)) - return arrow::schema({name, value}); + 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 diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index 854c4a25650..dca20dbeb81 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -20,24 +20,15 @@ #pragma once -#include #include #include #include -#include "arrow/flight/flight_sql/example/sqlite_statement.h" -#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" #include "arrow/flight/flight_sql/server.h" -#include "arrow/flight/flight_sql/sql_info_util.h" +#include "arrow/flight/flight_sql/sql_info_internal.h" #include "arrow/flight/server.h" #include "arrow/util/optional.h" -using string_list_t = std::vector; -using int32_to_int32_list_t = std::unordered_map>; -using SqlInfoResult = boost::variant; -using sql_info_id_to_result_t = std::unordered_map; - namespace arrow { namespace flight { namespace sql { @@ -120,8 +111,8 @@ struct ActionCreatePreparedStatementResult { std::string prepared_statement_handle; }; -/// \brief A free function responsible to create a ticket for the statement operation. -/// It is used by the server implementation. +/// \brief A utility function to create a ticket 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( @@ -129,7 +120,7 @@ arrow::Result CreateStatementQueryTicket( class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { private: - sql_info_id_to_result_t sql_info_id_to_result_; + SqlInfoResultMap sql_info_id_to_result_; public: Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/flight_sql/server_test.cc index 3dd78007893..d99209b898b 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/flight_sql/server_test.cc @@ -38,9 +38,9 @@ namespace arrow { namespace flight { namespace sql { -/// \brief Auxiliary boost::variant visitor used to assert that GetSqlInfo's values are +/// \brief Auxiliary variant visitor used to assert that GetSqlInfo's values are /// correctly placed on its DenseUnionArray -class SqlInfoDenseUnionValidator : public boost::static_visitor { +class SqlInfoDenseUnionValidator { private: const DenseUnionScalar& data; @@ -70,7 +70,7 @@ class SqlInfoDenseUnionValidator : public boost::static_visitor { } /// \brief Asserts that the current DenseUnionScalar equals to given string list - void operator()(const string_list_t& string_list) const { + void operator()(const std::vector& string_list) const { const auto& array = dynamic_cast( *(dynamic_cast(*data.value).value)); @@ -83,7 +83,8 @@ class SqlInfoDenseUnionValidator : public boost::static_visitor { /// \brief Asserts that the current DenseUnionScalar equals to given int32 to int32 list /// map. - void operator()(const int32_to_int32_list_t& int32_to_int32_list) const { + void operator()(const std::unordered_map>& + int32_to_int32_list) const { const auto& struct_array = dynamic_cast( *dynamic_cast(*data.value).value); const auto& keys = dynamic_cast(*struct_array.field(0)); @@ -117,6 +118,10 @@ class SqlInfoDenseUnionValidator : public boost::static_visitor { } explicit SqlInfoDenseUnionValidator(const DenseUnionScalar& data) : data(data) {} + + SqlInfoDenseUnionValidator(const SqlInfoDenseUnionValidator&) = delete; + SqlInfoDenseUnionValidator(SqlInfoDenseUnionValidator&&) = delete; + SqlInfoDenseUnionValidator& operator=(const SqlInfoDenseUnionValidator&) = delete; }; std::unique_ptr server; @@ -168,11 +173,15 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { arrow::schema({arrow::field("id", int64()), arrow::field("keyName", utf8()), arrow::field("value", int64()), arrow::field("foreignId", int64())}); - std::shared_ptr id_array = ArrayFromJSON(int64(), R"([1, 2, 3])"); - std::shared_ptr keyname_array = - ArrayFromJSON(utf8(), R"(["one", "zero", "negative one"])"); - std::shared_ptr value_array = ArrayFromJSON(int64(), R"([1, 0, -1])"); - std::shared_ptr foreignId_array = ArrayFromJSON(int64(), R"([1, 1, 1])"); + std::shared_ptr id_array; + std::shared_ptr keyname_array; + std::shared_ptr value_array; + std::shared_ptr foreignId_array; + ArrayFromVector({1, 2, 3}, &id_array); + ArrayFromVector({"one", "zero", "negative one"}, + &keyname_array); + ArrayFromVector({1, 0, -1}, &value_array); + ArrayFromVector({1, 1, 1}, &foreignId_array); const std::shared_ptr
& expected_table = Table::Make( expected_schema, {id_array, keyname_array, value_array, foreignId_array}); @@ -734,7 +743,7 @@ TEST(TestFlightSqlServer, TestCommandGetCrossReference) { } TEST(TestFlightSqlServer, TestCommandGetSqlInfo) { - const auto& sql_info_expected_results = sql::example::GetSqlInfoIdToResult(); + 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) { @@ -762,7 +771,7 @@ TEST(TestFlightSqlServer, TestCommandGetSqlInfo) { reinterpret_cast(*scalar)); const auto& expected_result = sql_info_expected_results.at(col_name_chunk_data[row]); - boost::apply_visitor(validator, expected_result); + arrow::util::visit(validator, expected_result); } } } diff --git a/cpp/src/arrow/flight/flight_sql/sql_info_internal.cc b/cpp/src/arrow/flight/flight_sql/sql_info_internal.cc new file mode 100644 index 00000000000..2a19ccdddde --- /dev/null +++ b/cpp/src/arrow/flight/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/flight_sql/sql_info_internal.h" + +#include "arrow/api.h" +#include "arrow/buffer.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace internal { + +Status SqlInfoResultAppender::operator()(const std::string& value) { + ARROW_RETURN_NOT_OK(value_builder_.Append(STRING_VALUE_INDEX)); + 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(BOOL_VALUE_INDEX)); + 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(BIGINT_VALUE_INDEX)); + 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(INT32_BITMASK_INDEX)); + 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(STRING_LIST_INDEX)); + 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(INT32_TO_INT32_LIST_INDEX)); + 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(STRING_VALUE_INDEX))), + bool_value_builder_( + reinterpret_cast(value_builder_.child(BOOL_VALUE_INDEX))), + bigint_value_builder_( + reinterpret_cast(value_builder_.child(BIGINT_VALUE_INDEX))), + int32_bitmask_builder_( + reinterpret_cast(value_builder_.child(INT32_BITMASK_INDEX))), + string_list_builder_( + reinterpret_cast(value_builder_.child(STRING_LIST_INDEX))), + int32_to_int32_list_builder_(reinterpret_cast( + value_builder_.child(INT32_TO_INT32_LIST_INDEX))) {} + +} // namespace internal +} // namespace sql +} // namespace flight +} // namespace arrow diff --git a/cpp/src/arrow/flight/flight_sql/sql_info_util.h b/cpp/src/arrow/flight/flight_sql/sql_info_internal.h similarity index 68% rename from cpp/src/arrow/flight/flight_sql/sql_info_util.h rename to cpp/src/arrow/flight/flight_sql/sql_info_internal.h index 1e6464f25af..a5450aed029 100644 --- a/cpp/src/arrow/flight/flight_sql/sql_info_util.h +++ b/cpp/src/arrow/flight/flight_sql/sql_info_internal.h @@ -20,13 +20,17 @@ #pragma once -#include +#include -using string_list_t = std::vector; -using int32_to_int32_list_t = std::unordered_map>; -using SqlInfoResult = boost::variant; -using sql_info_id_to_result_t = std::unordered_map; +#include +#include + +#include "arrow/type_fwd.h" + +using SqlInfoResult = + arrow::util::Variant, + std::unordered_map>>; +using SqlInfoResultMap = std::unordered_map; namespace arrow { namespace flight { @@ -35,7 +39,7 @@ namespace internal { /// \brief Auxiliary class used to populate GetSqlInfo's DenseUnionArray with different /// data types. -class SqlInfoResultAppender : public boost::static_visitor { +class SqlInfoResultAppender { public: /// \brief Appends a string to the DenseUnionBuilder. /// \param[in] value Value to be appended. @@ -55,19 +59,31 @@ class SqlInfoResultAppender : public boost::static_visitor { /// \brief Appends a string list to the DenseUnionBuilder. /// \param[in] value Value to be appended. - Status operator()(const string_list_t& value); + Status operator()(const std::vector& value); /// \brief Appends a int32 to int32 list map to the DenseUnionBuilder. /// \param[in] value Value to be appended. - Status operator()(const int32_to_int32_list_t& value); + Status operator()(const std::unordered_map>& value); - /// \brief Creates a boost::variant visitor that appends data to given + /// \brief Creates 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 { STRING_VALUE_INDEX = 0, BOOL_VALUE_INDEX = 1, diff --git a/cpp/src/arrow/flight/flight_sql/sql_info_util.cc b/cpp/src/arrow/flight/flight_sql/sql_info_util.cc deleted file mode 100644 index e77ce0bf64f..00000000000 --- a/cpp/src/arrow/flight/flight_sql/sql_info_util.cc +++ /dev/null @@ -1,113 +0,0 @@ -// 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 - -#define PROPERTY_TO_OPTIONAL(COMMAND, PROPERTY) \ - COMMAND.has_##PROPERTY() ? util::make_optional(COMMAND.PROPERTY()) : util::nullopt; - -#define REGISTER_FIELD(name, type, nullable) \ - const auto& name = field(ARROW_STRINGIFY(name), type, nullable); - -#define REGISTER_FIELD_NOT_NULL(name, type) REGISTER_FIELD(name, type, false) - -namespace arrow { -namespace flight { -namespace sql { -namespace internal { - -Status SqlInfoResultAppender::operator()(const std::string& value) { - const int8_t type_index = STRING_VALUE_INDEX; - ARROW_RETURN_NOT_OK(value_builder_.Append(type_index)); - auto* string_value_builder = - reinterpret_cast(value_builder_.child(type_index)); - ARROW_RETURN_NOT_OK(string_value_builder->Append(value)); - return Status::OK(); -} - -Status SqlInfoResultAppender::operator()(const bool value) { - const int8_t type_index = BOOL_VALUE_INDEX; - ARROW_RETURN_NOT_OK(value_builder_.Append(type_index)); - auto* bool_value_builder = - reinterpret_cast(value_builder_.child(type_index)); - ARROW_RETURN_NOT_OK(bool_value_builder->Append(value)); - return Status::OK(); -} - -Status SqlInfoResultAppender::operator()(const int64_t value) { - const int8_t type_index = BIGINT_VALUE_INDEX; - ARROW_RETURN_NOT_OK(value_builder_.Append(type_index)); - auto* bigint_value_builder = - reinterpret_cast(value_builder_.child(type_index)); - ARROW_RETURN_NOT_OK(bigint_value_builder->Append(value)); - return Status::OK(); -} - -Status SqlInfoResultAppender::operator()(const int32_t value) { - const int8_t type_index = INT32_BITMASK_INDEX; - ARROW_RETURN_NOT_OK(value_builder_.Append(type_index)); - auto* int32_bitmask_builder = - reinterpret_cast(value_builder_.child(type_index)); - ARROW_RETURN_NOT_OK(int32_bitmask_builder->Append(value)); - return Status::OK(); -} - -Status SqlInfoResultAppender::operator()(const string_list_t& value) { - const int8_t type_index = STRING_LIST_INDEX; - ARROW_RETURN_NOT_OK(value_builder_.Append(type_index)); - auto* string_list_builder = - reinterpret_cast(value_builder_.child(type_index)); - 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 int32_to_int32_list_t& value) { - const int8_t type_index = INT32_TO_INT32_LIST_INDEX; - ARROW_RETURN_NOT_OK(value_builder_.Append(type_index)); - auto* int32_to_int32_list_builder = - reinterpret_cast(value_builder_.child(type_index)); - 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) {} - -} // namespace internal -} // namespace sql -} // namespace flight -} // namespace arrow diff --git a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc index 88b81667b52..9d89c86b251 100644 --- a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/flight_sql/test_app_cli.cc @@ -74,7 +74,7 @@ Status PrintResultsForEndpoint(FlightSqlClient& client, while (true) { ARROW_RETURN_NOT_OK(stream->Next(&chunk)); - if (chunk.data) { + if (chunk.data == nullptr) { break; } std::cout << chunk.data->ToString() << std::endl; From 9fcacf22c236ce45fd31ce13b0c4cf1370dee877 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 15 Nov 2021 18:13:57 -0300 Subject: [PATCH 219/237] Fix CMake minor issue, add sql_info_types.h --- .../arrow/flight/flight_sql/CMakeLists.txt | 2 - cpp/src/arrow/flight/flight_sql/server.cc | 1 + cpp/src/arrow/flight/flight_sql/server.h | 2 +- .../flight/flight_sql/sql_info_internal.h | 15 +------- .../arrow/flight/flight_sql/sql_info_types.h | 37 +++++++++++++++++++ 5 files changed, 40 insertions(+), 17 deletions(-) create mode 100644 cpp/src/arrow/flight/flight_sql/sql_info_types.h diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt index f17fccfc342..dcbb5cea1aa 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/flight_sql/CMakeLists.txt @@ -82,8 +82,6 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) find_package(SQLite3Alt REQUIRED) add_executable(flight_sql_test_server - server.cc - sql_info_internal.cc test_server_cli.cc example/sqlite_statement.cc example/sqlite_statement_batch_reader.cc diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/flight_sql/server.cc index 30eb5ac9d92..58742f0f070 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/flight_sql/server.cc @@ -25,6 +25,7 @@ #include "arrow/api.h" #include "arrow/buffer.h" #include "arrow/flight/flight_sql/FlightSql.pb.h" +#include "arrow/flight/flight_sql/sql_info_internal.h" #include "arrow/type.h" #include "arrow/util/logging.h" diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/flight_sql/server.h index dca20dbeb81..8ab589f056e 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/flight_sql/server.h @@ -25,7 +25,7 @@ #include #include "arrow/flight/flight_sql/server.h" -#include "arrow/flight/flight_sql/sql_info_internal.h" +#include "arrow/flight/flight_sql/sql_info_types.h" #include "arrow/flight/server.h" #include "arrow/util/optional.h" diff --git a/cpp/src/arrow/flight/flight_sql/sql_info_internal.h b/cpp/src/arrow/flight/flight_sql/sql_info_internal.h index a5450aed029..6d500e9ec8c 100644 --- a/cpp/src/arrow/flight/flight_sql/sql_info_internal.h +++ b/cpp/src/arrow/flight/flight_sql/sql_info_internal.h @@ -15,22 +15,9 @@ // 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/type_fwd.h" - -using SqlInfoResult = - arrow::util::Variant, - std::unordered_map>>; -using SqlInfoResultMap = std::unordered_map; +#include "arrow/flight/flight_sql/sql_info_types.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight_sql/sql_info_types.h b/cpp/src/arrow/flight/flight_sql/sql_info_types.h new file mode 100644 index 00000000000..a8b87fc1143 --- /dev/null +++ b/cpp/src/arrow/flight/flight_sql/sql_info_types.h @@ -0,0 +1,37 @@ +// 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/type_fwd.h" +#include "arrow/util/variant.h" + +namespace arrow { +namespace flight { +namespace sql { + +using SqlInfoResult = + arrow::util::Variant, + std::unordered_map>>; +using SqlInfoResultMap = std::unordered_map; + +} // namespace sql +} // namespace flight +} // namespace arrow From 56d84e9fd210a74c0aea7fb4675b76e35e12de76 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 19 Nov 2021 14:27:50 -0300 Subject: [PATCH 220/237] Rename directory flight_sql to sql (#210) --- cpp/cmake_modules/FindArrowFlightSql.cmake | 2 +- cpp/src/arrow/CMakeLists.txt | 2 +- .../{flight_sql => sql}/ArrowFlightSqlConfig.cmake.in | 0 .../arrow/flight/{flight_sql => sql}/CMakeLists.txt | 4 ++-- cpp/src/arrow/flight/{flight_sql => sql}/api.h | 2 +- .../flight/{flight_sql => sql}/arrow-flight-sql.pc.in | 0 cpp/src/arrow/flight/{flight_sql => sql}/client.cc | 4 ++-- cpp/src/arrow/flight/{flight_sql => sql}/client.h | 0 .../arrow/flight/{flight_sql => sql}/client_test.cc | 4 ++-- .../{flight_sql => sql}/example/sqlite_server.cc | 10 +++++----- .../flight/{flight_sql => sql}/example/sqlite_server.h | 8 ++++---- .../{flight_sql => sql}/example/sqlite_statement.cc | 4 ++-- .../{flight_sql => sql}/example/sqlite_statement.h | 0 .../example/sqlite_statement_batch_reader.cc | 4 ++-- .../example/sqlite_statement_batch_reader.h | 2 +- .../example/sqlite_tables_schema_batch_reader.cc | 8 ++++---- .../example/sqlite_tables_schema_batch_reader.h | 4 ++-- cpp/src/arrow/flight/{flight_sql => sql}/server.cc | 6 +++--- cpp/src/arrow/flight/{flight_sql => sql}/server.h | 4 ++-- .../arrow/flight/{flight_sql => sql}/server_test.cc | 8 ++++---- .../flight/{flight_sql => sql}/sql_info_internal.cc | 2 +- .../flight/{flight_sql => sql}/sql_info_internal.h | 2 +- .../arrow/flight/{flight_sql => sql}/sql_info_types.h | 0 .../arrow/flight/{flight_sql => sql}/test_app_cli.cc | 2 +- .../flight/{flight_sql => sql}/test_server_cli.cc | 2 +- 25 files changed, 42 insertions(+), 42 deletions(-) rename cpp/src/arrow/flight/{flight_sql => sql}/ArrowFlightSqlConfig.cmake.in (100%) rename cpp/src/arrow/flight/{flight_sql => sql}/CMakeLists.txt (96%) rename cpp/src/arrow/flight/{flight_sql => sql}/api.h (94%) rename cpp/src/arrow/flight/{flight_sql => sql}/arrow-flight-sql.pc.in (100%) rename cpp/src/arrow/flight/{flight_sql => sql}/client.cc (99%) rename cpp/src/arrow/flight/{flight_sql => sql}/client.h (100%) rename cpp/src/arrow/flight/{flight_sql => sql}/client_test.cc (99%) rename cpp/src/arrow/flight/{flight_sql => sql}/example/sqlite_server.cc (98%) rename cpp/src/arrow/flight/{flight_sql => sql}/example/sqlite_server.h (98%) rename cpp/src/arrow/flight/{flight_sql => sql}/example/sqlite_statement.cc (97%) rename cpp/src/arrow/flight/{flight_sql => sql}/example/sqlite_statement.h (100%) rename cpp/src/arrow/flight/{flight_sql => sql}/example/sqlite_statement_batch_reader.cc (98%) rename cpp/src/arrow/flight/{flight_sql => sql}/example/sqlite_statement_batch_reader.h (97%) rename cpp/src/arrow/flight/{flight_sql => sql}/example/sqlite_tables_schema_batch_reader.cc (93%) rename cpp/src/arrow/flight/{flight_sql => sql}/example/sqlite_tables_schema_batch_reader.h (93%) rename cpp/src/arrow/flight/{flight_sql => sql}/server.cc (99%) rename cpp/src/arrow/flight/{flight_sql => sql}/server.h (99%) rename cpp/src/arrow/flight/{flight_sql => sql}/server_test.cc (99%) rename cpp/src/arrow/flight/{flight_sql => sql}/sql_info_internal.cc (98%) rename cpp/src/arrow/flight/{flight_sql => sql}/sql_info_internal.h (98%) rename cpp/src/arrow/flight/{flight_sql => sql}/sql_info_types.h (100%) rename cpp/src/arrow/flight/{flight_sql => sql}/test_app_cli.cc (99%) rename cpp/src/arrow/flight/{flight_sql => sql}/test_server_cli.cc (97%) diff --git a/cpp/cmake_modules/FindArrowFlightSql.cmake b/cpp/cmake_modules/FindArrowFlightSql.cmake index 35141749b1f..cbca81cac44 100644 --- a/cpp/cmake_modules/FindArrowFlightSql.cmake +++ b/cpp/cmake_modules/FindArrowFlightSql.cmake @@ -51,7 +51,7 @@ if(ARROW_FOUND) arrow_find_package(ARROW_FLIGHT_SQL "${ARROW_HOME}" arrow_flight_sql - arrow/flight/flight_sql/api.h + arrow/flight/sql/api.h ArrowFlightSql arrow-flight-sql) if(NOT ARROW_FLIGHT_SQL_VERSION) diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 01b646d4e76..502629a92a4 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -733,7 +733,7 @@ if(ARROW_FLIGHT) endif() if(ARROW_FLIGHT_SQL) - add_subdirectory(flight/flight_sql) + add_subdirectory(flight/sql) endif() if(ARROW_HIVESERVER2) diff --git a/cpp/src/arrow/flight/flight_sql/ArrowFlightSqlConfig.cmake.in b/cpp/src/arrow/flight/sql/ArrowFlightSqlConfig.cmake.in similarity index 100% rename from cpp/src/arrow/flight/flight_sql/ArrowFlightSqlConfig.cmake.in rename to cpp/src/arrow/flight/sql/ArrowFlightSqlConfig.cmake.in diff --git a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt b/cpp/src/arrow/flight/sql/CMakeLists.txt similarity index 96% rename from cpp/src/arrow/flight/flight_sql/CMakeLists.txt rename to cpp/src/arrow/flight/sql/CMakeLists.txt index dcbb5cea1aa..656b1123dcf 100644 --- a/cpp/src/arrow/flight/flight_sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/sql/CMakeLists.txt @@ -17,7 +17,7 @@ add_custom_target(arrow_flight_sql) -arrow_install_all_headers("arrow/flight/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) @@ -49,7 +49,7 @@ add_arrow_lib(arrow_flight_sql SOURCES ${ARROW_FLIGHT_SQL_SRCS} PRECOMPILED_HEADERS - "$<$:arrow/flight/flight_sql/pch.h>" + "$<$:arrow/flight/sql/pch.h>" DEPENDENCIES flight_sql_protobuf_gen SHARED_LINK_FLAGS diff --git a/cpp/src/arrow/flight/flight_sql/api.h b/cpp/src/arrow/flight/sql/api.h similarity index 94% rename from cpp/src/arrow/flight/flight_sql/api.h rename to cpp/src/arrow/flight/sql/api.h index 8d5124ee86d..3b909eedf29 100644 --- a/cpp/src/arrow/flight/flight_sql/api.h +++ b/cpp/src/arrow/flight/sql/api.h @@ -17,4 +17,4 @@ #pragma once -#include "arrow/flight/flight_sql/client.h" +#include "arrow/flight/sql/client.h" diff --git a/cpp/src/arrow/flight/flight_sql/arrow-flight-sql.pc.in b/cpp/src/arrow/flight/sql/arrow-flight-sql.pc.in similarity index 100% rename from cpp/src/arrow/flight/flight_sql/arrow-flight-sql.pc.in rename to cpp/src/arrow/flight/sql/arrow-flight-sql.pc.in diff --git a/cpp/src/arrow/flight/flight_sql/client.cc b/cpp/src/arrow/flight/sql/client.cc similarity index 99% rename from cpp/src/arrow/flight/flight_sql/client.cc rename to cpp/src/arrow/flight/sql/client.cc index e565c872cee..9a800b256af 100644 --- a/cpp/src/arrow/flight/flight_sql/client.cc +++ b/cpp/src/arrow/flight/sql/client.cc @@ -15,12 +15,12 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight_sql/client.h" +#include "arrow/flight/sql/client.h" #include #include "arrow/buffer.h" -#include "arrow/flight/flight_sql/FlightSql.pb.h" +#include "arrow/flight/sql/FlightSql.pb.h" #include "arrow/flight/types.h" #include "arrow/io/memory.h" #include "arrow/ipc/reader.h" diff --git a/cpp/src/arrow/flight/flight_sql/client.h b/cpp/src/arrow/flight/sql/client.h similarity index 100% rename from cpp/src/arrow/flight/flight_sql/client.h rename to cpp/src/arrow/flight/sql/client.h diff --git a/cpp/src/arrow/flight/flight_sql/client_test.cc b/cpp/src/arrow/flight/sql/client_test.cc similarity index 99% rename from cpp/src/arrow/flight/flight_sql/client_test.cc rename to cpp/src/arrow/flight/sql/client_test.cc index 0397ff91489..0c6ec1c6cd6 100644 --- a/cpp/src/arrow/flight/flight_sql/client_test.cc +++ b/cpp/src/arrow/flight/sql/client_test.cc @@ -23,8 +23,8 @@ #include -#include "arrow/flight/flight_sql/FlightSql.pb.h" -#include "arrow/flight/flight_sql/api.h" +#include "arrow/flight/sql/FlightSql.pb.h" +#include "arrow/flight/sql/api.h" #include "arrow/testing/gtest_util.h" namespace pb = arrow::flight::protocol; diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc b/cpp/src/arrow/flight/sql/example/sqlite_server.cc similarity index 98% rename from cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc rename to cpp/src/arrow/flight/sql/example/sqlite_server.cc index 6d150bd814b..8854b9c5311 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight_sql/example/sqlite_server.h" +#include "arrow/flight/sql/example/sqlite_server.h" #include @@ -24,10 +24,10 @@ #include #include "arrow/api.h" -#include "arrow/flight/flight_sql/example/sqlite_statement.h" -#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" -#include "arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h" -#include "arrow/flight/flight_sql/server.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 { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h b/cpp/src/arrow/flight/sql/example/sqlite_server.h similarity index 98% rename from cpp/src/arrow/flight/flight_sql/example/sqlite_server.h rename to cpp/src/arrow/flight/sql/example/sqlite_server.h index ec126cf4296..6eb890d5ddf 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.h @@ -26,10 +26,10 @@ #include #include "arrow/api.h" -#include "arrow/flight/flight_sql/FlightSql.pb.h" -#include "arrow/flight/flight_sql/example/sqlite_statement.h" -#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" -#include "arrow/flight/flight_sql/server.h" +#include "arrow/flight/sql/FlightSql.pb.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 { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc b/cpp/src/arrow/flight/sql/example/sqlite_statement.cc similarity index 97% rename from cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc rename to cpp/src/arrow/flight/sql/example/sqlite_statement.cc index c08323bb21a..3d7c5efeb23 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_statement.cc @@ -15,13 +15,13 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/flight/sql/example/sqlite_statement.h" #include #include -#include "arrow/flight/flight_sql/example/sqlite_server.h" +#include "arrow/flight/sql/example/sqlite_server.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h b/cpp/src/arrow/flight/sql/example/sqlite_statement.h similarity index 100% rename from cpp/src/arrow/flight/flight_sql/example/sqlite_statement.h rename to cpp/src/arrow/flight/sql/example/sqlite_statement.h diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc b/cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.cc similarity index 98% rename from cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc rename to cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.cc index 4071c729a8a..c6ac34bfddb 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.cc @@ -15,12 +15,12 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" +#include "arrow/flight/sql/example/sqlite_statement_batch_reader.h" #include #include "arrow/builder.h" -#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/flight/sql/example/sqlite_statement.h" #define STRING_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ case TYPE_CLASS##Type::type_id: { \ diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h b/cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.h similarity index 97% rename from cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h rename to cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.h index 84064a8daaa..8a6bc6078e7 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h +++ b/cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.h @@ -21,7 +21,7 @@ #include -#include "arrow/flight/flight_sql/example/sqlite_statement.h" +#include "arrow/flight/sql/example/sqlite_statement.h" #include "arrow/record_batch.h" namespace arrow { diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc b/cpp/src/arrow/flight/sql/example/sqlite_tables_schema_batch_reader.cc similarity index 93% rename from cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc rename to cpp/src/arrow/flight/sql/example/sqlite_tables_schema_batch_reader.cc index d9f16dce94c..7fb68a709f8 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_tables_schema_batch_reader.cc @@ -15,15 +15,15 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h" +#include "arrow/flight/sql/example/sqlite_tables_schema_batch_reader.h" #include #include -#include "arrow/flight/flight_sql/example/sqlite_server.h" -#include "arrow/flight/flight_sql/example/sqlite_statement.h" -#include "arrow/flight/flight_sql/server.h" +#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" diff --git a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h b/cpp/src/arrow/flight/sql/example/sqlite_tables_schema_batch_reader.h similarity index 93% rename from cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h rename to cpp/src/arrow/flight/sql/example/sqlite_tables_schema_batch_reader.h index dce725e7a2b..ecba88efb2f 100644 --- a/cpp/src/arrow/flight/flight_sql/example/sqlite_tables_schema_batch_reader.h +++ b/cpp/src/arrow/flight/sql/example/sqlite_tables_schema_batch_reader.h @@ -22,8 +22,8 @@ #include #include -#include "arrow/flight/flight_sql/example/sqlite_statement.h" -#include "arrow/flight/flight_sql/example/sqlite_statement_batch_reader.h" +#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 { diff --git a/cpp/src/arrow/flight/flight_sql/server.cc b/cpp/src/arrow/flight/sql/server.cc similarity index 99% rename from cpp/src/arrow/flight/flight_sql/server.cc rename to cpp/src/arrow/flight/sql/server.cc index 58742f0f070..f07222a937e 100644 --- a/cpp/src/arrow/flight/flight_sql/server.cc +++ b/cpp/src/arrow/flight/sql/server.cc @@ -18,14 +18,14 @@ // Interfaces to use for defining Flight RPC servers. API should be considered // experimental for now -#include "arrow/flight/flight_sql/server.h" +#include "arrow/flight/sql/server.h" #include #include "arrow/api.h" #include "arrow/buffer.h" -#include "arrow/flight/flight_sql/FlightSql.pb.h" -#include "arrow/flight/flight_sql/sql_info_internal.h" +#include "arrow/flight/sql/FlightSql.pb.h" +#include "arrow/flight/sql/sql_info_internal.h" #include "arrow/type.h" #include "arrow/util/logging.h" diff --git a/cpp/src/arrow/flight/flight_sql/server.h b/cpp/src/arrow/flight/sql/server.h similarity index 99% rename from cpp/src/arrow/flight/flight_sql/server.h rename to cpp/src/arrow/flight/sql/server.h index 8ab589f056e..f1757fc27be 100644 --- a/cpp/src/arrow/flight/flight_sql/server.h +++ b/cpp/src/arrow/flight/sql/server.h @@ -24,9 +24,9 @@ #include #include -#include "arrow/flight/flight_sql/server.h" -#include "arrow/flight/flight_sql/sql_info_types.h" #include "arrow/flight/server.h" +#include "arrow/flight/sql/server.h" +#include "arrow/flight/sql/sql_info_types.h" #include "arrow/util/optional.h" namespace arrow { diff --git a/cpp/src/arrow/flight/flight_sql/server_test.cc b/cpp/src/arrow/flight/sql/server_test.cc similarity index 99% rename from cpp/src/arrow/flight/flight_sql/server_test.cc rename to cpp/src/arrow/flight/sql/server_test.cc index d99209b898b..a044f2ad4f2 100644 --- a/cpp/src/arrow/flight/flight_sql/server_test.cc +++ b/cpp/src/arrow/flight/sql/server_test.cc @@ -15,16 +15,16 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight_sql/server.h" +#include "arrow/flight/sql/server.h" #include #include #include "arrow/api.h" #include "arrow/flight/api.h" -#include "arrow/flight/flight_sql/FlightSql.pb.h" -#include "arrow/flight/flight_sql/api.h" -#include "arrow/flight/flight_sql/example/sqlite_server.h" +#include "arrow/flight/sql/FlightSql.pb.h" +#include "arrow/flight/sql/api.h" +#include "arrow/flight/sql/example/sqlite_server.h" #include "arrow/flight/test_util.h" #include "arrow/flight/types.h" #include "arrow/testing/gtest_util.h" diff --git a/cpp/src/arrow/flight/flight_sql/sql_info_internal.cc b/cpp/src/arrow/flight/sql/sql_info_internal.cc similarity index 98% rename from cpp/src/arrow/flight/flight_sql/sql_info_internal.cc rename to cpp/src/arrow/flight/sql/sql_info_internal.cc index 2a19ccdddde..ece963654aa 100644 --- a/cpp/src/arrow/flight/flight_sql/sql_info_internal.cc +++ b/cpp/src/arrow/flight/sql/sql_info_internal.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/flight/flight_sql/sql_info_internal.h" +#include "arrow/flight/sql/sql_info_internal.h" #include "arrow/api.h" #include "arrow/buffer.h" diff --git a/cpp/src/arrow/flight/flight_sql/sql_info_internal.h b/cpp/src/arrow/flight/sql/sql_info_internal.h similarity index 98% rename from cpp/src/arrow/flight/flight_sql/sql_info_internal.h rename to cpp/src/arrow/flight/sql/sql_info_internal.h index 6d500e9ec8c..4f08b109303 100644 --- a/cpp/src/arrow/flight/flight_sql/sql_info_internal.h +++ b/cpp/src/arrow/flight/sql/sql_info_internal.h @@ -17,7 +17,7 @@ #pragma once -#include "arrow/flight/flight_sql/sql_info_types.h" +#include "arrow/flight/sql/sql_info_types.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/flight_sql/sql_info_types.h b/cpp/src/arrow/flight/sql/sql_info_types.h similarity index 100% rename from cpp/src/arrow/flight/flight_sql/sql_info_types.h rename to cpp/src/arrow/flight/sql/sql_info_types.h diff --git a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc b/cpp/src/arrow/flight/sql/test_app_cli.cc similarity index 99% rename from cpp/src/arrow/flight/flight_sql/test_app_cli.cc rename to cpp/src/arrow/flight/sql/test_app_cli.cc index 9d89c86b251..5e3f9d7a660 100644 --- a/cpp/src/arrow/flight/flight_sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/sql/test_app_cli.cc @@ -24,7 +24,7 @@ #include "arrow/array/builder_binary.h" #include "arrow/array/builder_primitive.h" #include "arrow/flight/api.h" -#include "arrow/flight/flight_sql/api.h" +#include "arrow/flight/sql/api.h" #include "arrow/io/memory.h" #include "arrow/pretty_print.h" #include "arrow/status.h" diff --git a/cpp/src/arrow/flight/flight_sql/test_server_cli.cc b/cpp/src/arrow/flight/sql/test_server_cli.cc similarity index 97% rename from cpp/src/arrow/flight/flight_sql/test_server_cli.cc rename to cpp/src/arrow/flight/sql/test_server_cli.cc index 22c7610c8dd..8074ab534bd 100644 --- a/cpp/src/arrow/flight/flight_sql/test_server_cli.cc +++ b/cpp/src/arrow/flight/sql/test_server_cli.cc @@ -21,8 +21,8 @@ #include #include -#include "arrow/flight/flight_sql/example/sqlite_server.h" #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" From 7d74b7efce86f77fd1b42716cec4d6b7c3a3bd13 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 25 Nov 2021 11:15:37 -0300 Subject: [PATCH 221/237] [C++] Address ratification comments (round 4) (#212) * Fix minor issues on sql_info_internal * Change table_types parameter to be a pointer * Improve GetSqlInfo error in case of no info * Replace ArrayFromVector to ArrayFromJSON in most cases * Improve server_test assertions and code quality --- cpp/src/arrow/flight/sql/client.cc | 8 +- cpp/src/arrow/flight/sql/client.h | 8 +- cpp/src/arrow/flight/sql/client_test.cc | 2 +- cpp/src/arrow/flight/sql/server.cc | 28 +- cpp/src/arrow/flight/sql/server_test.cc | 363 +++++++----------- cpp/src/arrow/flight/sql/sql_info_internal.cc | 30 +- cpp/src/arrow/flight/sql/sql_info_internal.h | 30 +- cpp/src/arrow/flight/sql/test_app_cli.cc | 5 +- 8 files changed, 196 insertions(+), 278 deletions(-) diff --git a/cpp/src/arrow/flight/sql/client.cc b/cpp/src/arrow/flight/sql/client.cc index 9a800b256af..d0ed9b4c6d7 100644 --- a/cpp/src/arrow/flight/sql/client.cc +++ b/cpp/src/arrow/flight/sql/client.cc @@ -135,7 +135,7 @@ arrow::Result> FlightSqlClient::GetSchemas( arrow::Result> FlightSqlClient::GetTables( const FlightCallOptions& options, const std::string* catalog, const std::string* schema_filter_pattern, const std::string* table_filter_pattern, - bool include_schema, const std::vector& table_types) { + bool include_schema, const std::vector* table_types) { flight_sql_pb::CommandGetTables command; if (catalog != NULLPTR) { @@ -152,8 +152,10 @@ arrow::Result> FlightSqlClient::GetTables( command.set_include_schema(include_schema); - for (const std::string& table_type : table_types) { - command.add_table_types(table_type); + if (table_types != NULLPTR) { + for (const std::string& table_type : *table_types) { + command.add_table_types(table_type); + } } return GetFlightInfoForCommand(*this, options, command); diff --git a/cpp/src/arrow/flight/sql/client.h b/cpp/src/arrow/flight/sql/client.h index f53960d8d6b..b3986f1bf38 100644 --- a/cpp/src/arrow/flight/sql/client.h +++ b/cpp/src/arrow/flight/sql/client.h @@ -92,7 +92,7 @@ class ARROW_EXPORT FlightSqlClient { arrow::Result> GetTables( const FlightCallOptions& options, const std::string* catalog, const std::string* schema_filter_pattern, const std::string* table_filter_pattern, - bool include_schema, const std::vector& table_types); + 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. @@ -244,12 +244,12 @@ class ARROW_EXPORT PreparedStatement { /// \return Status. Status SetParameters(std::shared_ptr parameter_binding); - /// \brief Closes the prepared statement. - /// \param[in] options RPC-layer hints for this call. + /// \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 Checks if the prepared statement is closed. + /// \brief Check if the prepared statement is closed. /// \return The state of the prepared statement. bool IsClosed() const; }; diff --git a/cpp/src/arrow/flight/sql/client_test.cc b/cpp/src/arrow/flight/sql/client_test.cc index 0c6ec1c6cd6..4bb3e859a91 100644 --- a/cpp/src/arrow/flight/sql/client_test.cc +++ b/cpp/src/arrow/flight/sql/client_test.cc @@ -164,7 +164,7 @@ TEST_F(TestFlightSqlClient, TestGetTables) { ASSERT_OK(sql_client.GetTables(call_options, &catalog, &schema_filter_pattern, &table_name_filter_pattern, include_schema, - table_types)); + &table_types)); } TEST_F(TestFlightSqlClient, TestGetTableTypes) { diff --git a/cpp/src/arrow/flight/sql/server.cc b/cpp/src/arrow/flight/sql/server.cc index f07222a937e..f3f83bc8a5f 100644 --- a/cpp/src/arrow/flight/sql/server.cc +++ b/cpp/src/arrow/flight/sql/server.cc @@ -22,11 +22,12 @@ #include -#include "arrow/api.h" #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" #include "arrow/util/logging.h" #define PROPERTY_TO_OPTIONAL(COMMAND, PROPERTY) \ @@ -38,6 +39,9 @@ namespace sql { namespace pb = arrow::flight::protocol; +using arrow::internal::checked_cast; +using arrow::internal::checked_pointer_cast; + arrow::Result CreateStatementQueryTicket( const std::string& statement_handle) { protocol::sql::TicketStatementQuery ticket_statement_query; @@ -370,8 +374,7 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, pb::sql::DoPutUpdateResult result; result.set_record_count(record_count); - const std::shared_ptr& buffer = - Buffer::FromString(result.SerializeAsString()); + const auto buffer = Buffer::FromString(result.SerializeAsString()); ARROW_RETURN_NOT_OK(writer->WriteMetadata(*buffer)); return Status::OK(); @@ -389,8 +392,7 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, pb::sql::DoPutUpdateResult result; result.set_record_count(record_count); - const std::shared_ptr& buffer = - Buffer::FromString(result.SerializeAsString()); + const auto buffer = Buffer::FromString(result.SerializeAsString()); ARROW_RETURN_NOT_OK(writer->WriteMetadata(*buffer)); return Status::OK(); @@ -494,7 +496,7 @@ Status FlightSqlServerBase::GetFlightInfoSqlInfo(const ServerCallContext& contex const GetSqlInfo& command, const FlightDescriptor& descriptor, std::unique_ptr* info) { - assert(info != nullptr); + DCHECK(info); if (sql_info_id_to_result_.empty()) { return Status::NotImplemented("GetFlightInfoSqlInfo not implemented"); @@ -521,12 +523,12 @@ Status FlightSqlServerBase::DoGetSqlInfo(const ServerCallContext& context, MemoryPool* memory_pool = default_memory_pool(); UInt32Builder name_field_builder(memory_pool); std::unique_ptr value_field_builder; - const auto& value_field_type = std::static_pointer_cast( + 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( - reinterpret_cast(*value_field_builder)); + checked_cast(value_field_builder.get())); // Populate both name_field_builder and value_field_builder for each element // on command.info. @@ -534,14 +536,12 @@ Status FlightSqlServerBase::DoGetSqlInfo(const ServerCallContext& context, // a DenseUnionBuilder). The population for each data type is implemented on // internal::SqlInfoResultAppender. for (const auto& info : command.info) { - if (!sql_info_id_to_result_.count(info)) { - return Status::KeyError( - "Attempt to fetch unregistered/unsupported SqlInfo option: " + - std::to_string(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, sql_info_id_to_result_.at(info))); + ARROW_RETURN_NOT_OK(arrow::util::visit(sql_info_result_appender, it->second)); } std::shared_ptr name; diff --git a/cpp/src/arrow/flight/sql/server_test.cc b/cpp/src/arrow/flight/sql/server_test.cc index a044f2ad4f2..f9f8328132c 100644 --- a/cpp/src/arrow/flight/sql/server_test.cc +++ b/cpp/src/arrow/flight/sql/server_test.cc @@ -20,9 +20,7 @@ #include #include -#include "arrow/api.h" #include "arrow/flight/api.h" -#include "arrow/flight/sql/FlightSql.pb.h" #include "arrow/flight/sql/api.h" #include "arrow/flight/sql/example/sqlite_server.h" #include "arrow/flight/test_util.h" @@ -34,6 +32,8 @@ using ::testing::Ref; namespace pb = arrow::flight::protocol; +using arrow::internal::checked_cast; + namespace arrow { namespace flight { namespace sql { @@ -47,32 +47,32 @@ class SqlInfoDenseUnionValidator { public: /// \brief Asserts that the current DenseUnionScalar equals to given string value void operator()(const std::string& string_value) const { - const auto& scalar = dynamic_cast(*data.value); + 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 = dynamic_cast(*data.value); + 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 = dynamic_cast(*data.value); + 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 = dynamic_cast(*data.value); + 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 = dynamic_cast( - *(dynamic_cast(*data.value).value)); + const auto& array = checked_cast( + *(checked_cast(*data.value).value)); ASSERT_EQ(string_list.size(), array.length()); @@ -85,10 +85,10 @@ class SqlInfoDenseUnionValidator { /// map. void operator()(const std::unordered_map>& int32_to_int32_list) const { - const auto& struct_array = dynamic_cast( - *dynamic_cast(*data.value).value); - const auto& keys = dynamic_cast(*struct_array.field(0)); - const auto& values = dynamic_cast(*struct_array.field(1)); + 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()); @@ -96,7 +96,7 @@ class SqlInfoDenseUnionValidator { // 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 = dynamic_cast(*key_scalar).value; + 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)); @@ -111,7 +111,7 @@ class SqlInfoDenseUnionValidator { 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 = dynamic_cast(*list_item_scalar).value; + const auto& list_item = checked_cast(*list_item_scalar).value; ASSERT_EQ(expected_int32_list[j], list_item); } } @@ -173,15 +173,10 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { arrow::schema({arrow::field("id", int64()), arrow::field("keyName", utf8()), arrow::field("value", int64()), arrow::field("foreignId", int64())}); - std::shared_ptr id_array; - std::shared_ptr keyname_array; - std::shared_ptr value_array; - std::shared_ptr foreignId_array; - ArrayFromVector({1, 2, 3}, &id_array); - ArrayFromVector({"one", "zero", "negative one"}, - &keyname_array); - ArrayFromVector({1, 0, -1}, &value_array); - ArrayFromVector({1, 1, 1}, &foreignId_array); + const auto id_array = ArrayFromJSON(int64(), R"([1, 2, 3])"); + const auto keyname_array = ArrayFromJSON(utf8(), R"(["one", "zero", "negative one"])"); + const auto value_array = ArrayFromJSON(int64(), R"([1, 0, -1])"); + const auto foreignId_array = ArrayFromJSON(int64(), R"([1, 1, 1])"); const std::shared_ptr
& expected_table = Table::Make( expected_schema, {id_array, keyname_array, value_array, foreignId_array}); @@ -190,10 +185,8 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { } TEST(TestFlightSqlServer, TestCommandGetTables) { - std::vector table_types; - ASSERT_OK_AND_ASSIGN( - auto flight_info, - sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types)); + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetTables({}, nullptr, nullptr, + nullptr, false, nullptr)); ASSERT_OK_AND_ASSIGN(auto stream, sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); @@ -203,11 +196,10 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { ASSERT_OK_AND_ASSIGN(auto catalog_name, MakeArrayOfNull(utf8(), 3)) ASSERT_OK_AND_ASSIGN(auto schema_name, MakeArrayOfNull(utf8(), 3)) - std::shared_ptr table_name; - ArrayFromVector( - {"foreignTable", "intTable", "sqlite_sequence"}, &table_name); - std::shared_ptr table_type; - ArrayFromVector({"table", "table", "table"}, &table_type); + + 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}); @@ -216,12 +208,10 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { } TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { - std::vector table_types; - std::string table_filter_pattern = "int%"; - ASSERT_OK_AND_ASSIGN(auto flight_info, - sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, - false, table_types)); + ASSERT_OK_AND_ASSIGN( + auto flight_info, + sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, false, nullptr)); ASSERT_OK_AND_ASSIGN(auto stream, sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); @@ -229,12 +219,10 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - ASSERT_OK_AND_ASSIGN(auto catalog_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto schema_name, MakeArrayOfNull(utf8(), 1)) - std::shared_ptr table_name; - std::shared_ptr table_type; - ArrayFromVector({"intTable"}, &table_name); - ArrayFromVector({"table"}, &table_type); + 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}); @@ -247,7 +235,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { ASSERT_OK_AND_ASSIGN( auto flight_info, - sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types)); + sql_client->GetTables({}, nullptr, nullptr, nullptr, false, &table_types)); ASSERT_OK_AND_ASSIGN(auto stream, sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); @@ -255,7 +243,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - ASSERT_TRUE(table->schema()->Equals(SqlSchema::GetTablesSchema())); + AssertSchemaEqual(SqlSchema::GetTablesSchema(), table->schema()); ASSERT_EQ(table->num_rows(), 0); } @@ -265,7 +253,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { ASSERT_OK_AND_ASSIGN( auto flight_info, - sql_client->GetTables({}, nullptr, nullptr, nullptr, false, table_types)); + sql_client->GetTables({}, nullptr, nullptr, nullptr, false, &table_types)); ASSERT_OK_AND_ASSIGN(auto stream, sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); @@ -273,18 +261,11 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { 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)) - std::shared_ptr table_name; - ArrayFromVector( - { - "foreignTable", - "intTable", - "sqlite_sequence", - }, - &table_name); - std::shared_ptr table_type; - ArrayFromVector({"table", "table", "table"}, &table_type); + 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}); @@ -293,12 +274,10 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { } TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { - std::vector table_types; - std::string table_filter_pattern = "int%"; - ASSERT_OK_AND_ASSIGN(auto flight_info, - sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, - true, table_types)); + ASSERT_OK_AND_ASSIGN( + auto flight_info, + sql_client->GetTables({}, nullptr, nullptr, &table_filter_pattern, true, nullptr)); ASSERT_OK_AND_ASSIGN(auto stream, sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); @@ -306,24 +285,18 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - ASSERT_OK_AND_ASSIGN(auto catalog_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto schema_name, MakeArrayOfNull(utf8(), 1)) - std::shared_ptr table_name; - std::shared_ptr table_type; - ArrayFromVector({"intTable"}, &table_name); - ArrayFromVector({"table"}, &table_type); + 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)}); - const arrow::Result>& value = - ipc::SerializeSchema(*schema_table); - std::shared_ptr schema_buffer; + ASSERT_OK_AND_ASSIGN(auto schema_buffer, ipc::SerializeSchema(*schema_table)); - ASSERT_OK_AND_ASSIGN(schema_buffer, value); std::shared_ptr table_schema; - ArrayFromVector({schema_buffer->ToString()}, &table_schema); const std::shared_ptr
& expected_table = @@ -344,7 +317,7 @@ TEST(TestFlightSqlServer, TestCommandGetCatalogs) { const std::shared_ptr& expected_schema = SqlSchema::GetCatalogsSchema(); - ASSERT_TRUE(table->schema()->Equals(*expected_schema)); + AssertSchemaEqual(expected_schema, table->schema()); ASSERT_EQ(0, table->num_rows()); } @@ -359,7 +332,7 @@ TEST(TestFlightSqlServer, TestCommandGetSchemas) { const std::shared_ptr& expected_schema = SqlSchema::GetSchemasSchema(); - ASSERT_TRUE(table->schema()->Equals(*expected_schema)); + AssertSchemaEqual(expected_schema, table->schema()); ASSERT_EQ(0, table->num_rows()); } @@ -372,8 +345,7 @@ TEST(TestFlightSqlServer, TestCommandGetTableTypes) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - std::shared_ptr table_type; - ArrayFromVector({"table"}, &table_type); + const auto table_type = ArrayFromJSON(utf8(), R"(["table"])"); const std::shared_ptr
& expected_table = Table::Make(SqlSchema::GetTableTypesSchema(), {table_type}); @@ -417,15 +389,10 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQuery) { arrow::schema({arrow::field("id", int64()), arrow::field("keyName", utf8()), arrow::field("value", int64()), arrow::field("foreignId", int64())}); - std::shared_ptr id_array; - std::shared_ptr keyname_array; - std::shared_ptr value_array; - std::shared_ptr foreignId_array; - ArrayFromVector({1, 2, 3}, &id_array); - ArrayFromVector({"one", "zero", "negative one"}, - &keyname_array); - ArrayFromVector({1, 0, -1}, &value_array); - ArrayFromVector({1, 1, 1}, &foreignId_array); + const auto id_array = ArrayFromJSON(int64(), R"([1, 2, 3])"); + const auto keyname_array = ArrayFromJSON(utf8(), R"(["one", "zero", "negative one"])"); + const auto value_array = ArrayFromJSON(int64(), R"([1, 0, -1])"); + const auto foreignId_array = ArrayFromJSON(int64(), R"([1, 1, 1])"); const std::shared_ptr
& expected_table = Table::Make( expected_schema, {id_array, keyname_array, value_array, foreignId_array}); @@ -443,21 +410,14 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) const std::shared_ptr& expected_parameter_schema = arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); - ASSERT_TRUE(expected_parameter_schema->Equals(*parameter_schema)); + AssertSchemaEqual(expected_parameter_schema, parameter_schema); - std::shared_ptr type_ids; - ArrayFromVector({0}, &type_ids); - std::shared_ptr offsets; - ArrayFromVector({0}, &offsets); - - std::shared_ptr string_array; - ArrayFromVector({"%one"}, &string_array); - std::shared_ptr bytes_array; - ArrayFromVector({}, &bytes_array); - std::shared_ptr bigint_array; - ArrayFromVector({}, &bigint_array); - std::shared_ptr double_array; - ArrayFromVector({}, &double_array); + 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, @@ -482,14 +442,10 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) arrow::schema({arrow::field("id", int64()), arrow::field("keyName", utf8()), arrow::field("value", int64()), arrow::field("foreignId", int64())}); - std::shared_ptr id_array; - std::shared_ptr keyname_array; - std::shared_ptr value_array; - std::shared_ptr foreignId_array; - ArrayFromVector({1, 3}, &id_array); - ArrayFromVector({"one", "negative one"}, &keyname_array); - ArrayFromVector({1, -1}, &value_array); - ArrayFromVector({1, 1}, &foreignId_array); + 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}); @@ -523,21 +479,14 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding const std::shared_ptr& expected_parameter_schema = arrow::schema({arrow::field("parameter_1", example::GetUnknownColumnDataType())}); - ASSERT_TRUE(expected_parameter_schema->Equals(*parameter_schema)); - - std::shared_ptr type_ids; - ArrayFromVector({2}, &type_ids); - std::shared_ptr offsets; - ArrayFromVector({0}, &offsets); + AssertSchemaEqual(expected_parameter_schema, parameter_schema); - std::shared_ptr string_array; - ArrayFromVector({}, &string_array); - std::shared_ptr bytes_array; - ArrayFromVector({}, &bytes_array); - std::shared_ptr bigint_array; - ArrayFromVector({999}, &bigint_array); - std::shared_ptr double_array; - ArrayFromVector({}, &double_array); + 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, @@ -550,23 +499,16 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding ASSERT_OK(prepared_statement->SetParameters(record_batch)); - int64_t result; - ASSERT_OK_AND_ASSIGN(result, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); - ASSERT_EQ(3, result); + ASSERT_OK_AND_EQ(3, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); - ASSERT_OK_AND_ASSIGN(result, prepared_statement->ExecuteUpdate()); - ASSERT_EQ(1, result); + ASSERT_OK_AND_EQ(1, prepared_statement->ExecuteUpdate()); - ASSERT_OK_AND_ASSIGN(result, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); - ASSERT_EQ(4, result); + ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); - ASSERT_OK_AND_ASSIGN( - result, - sql_client->ExecuteUpdate({}, "DELETE FROM intTable WHERE keyName = 'new_value'")); - ASSERT_EQ(1, result); + ASSERT_OK_AND_EQ(1, sql_client->ExecuteUpdate( + {}, "DELETE FROM intTable WHERE keyName = 'new_value'")); - ASSERT_OK_AND_ASSIGN(result, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); - ASSERT_EQ(3, result); + ASSERT_OK_AND_EQ(3, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); } TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { @@ -575,23 +517,16 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { sql_client->Prepare( {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', 999)")); - int64_t result; - ASSERT_OK_AND_ASSIGN(result, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); - ASSERT_EQ(3, result); + ASSERT_OK_AND_EQ(3, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); - ASSERT_OK_AND_ASSIGN(result, prepared_statement->ExecuteUpdate()); - ASSERT_EQ(1, result); + ASSERT_OK_AND_EQ(1, prepared_statement->ExecuteUpdate()); - ASSERT_OK_AND_ASSIGN(result, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); - ASSERT_EQ(4, result); + ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); - ASSERT_OK_AND_ASSIGN( - result, - sql_client->ExecuteUpdate({}, "DELETE FROM intTable WHERE keyName = 'new_value'")); - ASSERT_EQ(1, result); + ASSERT_OK_AND_EQ(1, sql_client->ExecuteUpdate( + {}, "DELETE FROM intTable WHERE keyName = 'new_value'")); - ASSERT_OK_AND_ASSIGN(result, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); - ASSERT_EQ(3, result); + ASSERT_OK_AND_EQ(3, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); } TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { @@ -604,16 +539,12 @@ TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - ASSERT_OK_AND_ASSIGN(auto catalog_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto schema_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto key_name, MakeArrayOfNull(utf8(), 1)) - - std::shared_ptr table_name; - std::shared_ptr column_name; - std::shared_ptr key_sequence; - ArrayFromVector({"intTable"}, &table_name); - ArrayFromVector({"id"}, &column_name); - ArrayFromVector({1}, &key_sequence); + 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(), @@ -632,27 +563,19 @@ TEST(TestFlightSqlServer, TestCommandGetImportedKeys) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - std::shared_ptr pk_table_name; - std::shared_ptr pk_column_name; - std::shared_ptr fk_table_name; - std::shared_ptr fk_column_name; - std::shared_ptr key_sequence; - std::shared_ptr update_rule; - std::shared_ptr delete_rule; - ArrayFromVector({"foreignTable"}, &pk_table_name); - ArrayFromVector({"id"}, &pk_column_name); - ArrayFromVector({"intTable"}, &fk_table_name); - ArrayFromVector({"foreignId"}, &fk_column_name); - ArrayFromVector({0}, &key_sequence); - ArrayFromVector({3}, &update_rule); - ArrayFromVector({3}, &delete_rule); - - ASSERT_OK_AND_ASSIGN(auto pk_catalog_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto pk_schema_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto fk_catalog_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto fk_schema_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto fk_key_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto pk_key_name, MakeArrayOfNull(utf8(), 1)) + 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(), @@ -672,27 +595,19 @@ TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - std::shared_ptr pk_table_name; - std::shared_ptr pk_column_name; - std::shared_ptr fk_table_name; - std::shared_ptr fk_column_name; - std::shared_ptr key_sequence; - std::shared_ptr update_rule; - std::shared_ptr delete_rule; - ArrayFromVector({"foreignTable"}, &pk_table_name); - ArrayFromVector({"id"}, &pk_column_name); - ArrayFromVector({"intTable"}, &fk_table_name); - ArrayFromVector({"foreignId"}, &fk_column_name); - ArrayFromVector({0}, &key_sequence); - ArrayFromVector({3}, &update_rule); - ArrayFromVector({3}, &delete_rule); - - ASSERT_OK_AND_ASSIGN(auto pk_catalog_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto pk_schema_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto fk_catalog_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto fk_schema_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto fk_key_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto pk_key_name, MakeArrayOfNull(utf8(), 1)) + 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(), @@ -713,27 +628,20 @@ TEST(TestFlightSqlServer, TestCommandGetCrossReference) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - std::shared_ptr pk_table_name; - std::shared_ptr pk_column_name; - std::shared_ptr fk_table_name; - std::shared_ptr fk_column_name; - std::shared_ptr key_sequence; - std::shared_ptr update_rule; - std::shared_ptr delete_rule; - ArrayFromVector({"foreignTable"}, &pk_table_name); - ArrayFromVector({"id"}, &pk_column_name); - ArrayFromVector({"intTable"}, &fk_table_name); - ArrayFromVector({"foreignId"}, &fk_column_name); - ArrayFromVector({0}, &key_sequence); - ArrayFromVector({3}, &update_rule); - ArrayFromVector({3}, &delete_rule); - - ASSERT_OK_AND_ASSIGN(auto pk_catalog_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto pk_schema_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto fk_catalog_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto fk_schema_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto fk_key_name, MakeArrayOfNull(utf8(), 1)) - ASSERT_OK_AND_ASSIGN(auto pk_key_name, MakeArrayOfNull(utf8(), 1)) + 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, @@ -776,6 +684,17 @@ TEST(TestFlightSqlServer, TestCommandGetSqlInfo) { } } +TEST(TestFlightSqlServer, TestCommandGetSqlInfoNoInfo) { + FlightCallOptions call_options; + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetSqlInfo(call_options, {999999})); + auto result = sql_client->DoGet(call_options, flight_info->endpoints()[0].ticket); + + ASSERT_FALSE(result.ok()); + ASSERT_TRUE(result.status().IsKeyError()); + ASSERT_THAT(result.status().message(), + ::testing::HasSubstr("No information for SQL info number 999999.")); +} + auto env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); } // namespace sql diff --git a/cpp/src/arrow/flight/sql/sql_info_internal.cc b/cpp/src/arrow/flight/sql/sql_info_internal.cc index ece963654aa..74718fb7cb5 100644 --- a/cpp/src/arrow/flight/sql/sql_info_internal.cc +++ b/cpp/src/arrow/flight/sql/sql_info_internal.cc @@ -17,8 +17,8 @@ #include "arrow/flight/sql/sql_info_internal.h" -#include "arrow/api.h" #include "arrow/buffer.h" +#include "arrow/builder.h" namespace arrow { namespace flight { @@ -26,31 +26,31 @@ namespace sql { namespace internal { Status SqlInfoResultAppender::operator()(const std::string& value) { - ARROW_RETURN_NOT_OK(value_builder_.Append(STRING_VALUE_INDEX)); + 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(BOOL_VALUE_INDEX)); + 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(BIGINT_VALUE_INDEX)); + 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(INT32_BITMASK_INDEX)); + 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(STRING_LIST_INDEX)); + 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()); @@ -62,7 +62,7 @@ Status SqlInfoResultAppender::operator()(const std::vector& value) Status SqlInfoResultAppender::operator()( const std::unordered_map>& value) { - ARROW_RETURN_NOT_OK(value_builder_.Append(INT32_TO_INT32_LIST_INDEX)); + 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( @@ -80,20 +80,20 @@ Status SqlInfoResultAppender::operator()( return Status::OK(); } -SqlInfoResultAppender::SqlInfoResultAppender(DenseUnionBuilder& value_builder) +SqlInfoResultAppender::SqlInfoResultAppender(DenseUnionBuilder* value_builder) : value_builder_(value_builder), string_value_builder_( - reinterpret_cast(value_builder_.child(STRING_VALUE_INDEX))), + reinterpret_cast(value_builder_->child(kStringValueIndex))), bool_value_builder_( - reinterpret_cast(value_builder_.child(BOOL_VALUE_INDEX))), + reinterpret_cast(value_builder_->child(kBoolValueIndex))), bigint_value_builder_( - reinterpret_cast(value_builder_.child(BIGINT_VALUE_INDEX))), + reinterpret_cast(value_builder_->child(kBigIntValueIndex))), int32_bitmask_builder_( - reinterpret_cast(value_builder_.child(INT32_BITMASK_INDEX))), + reinterpret_cast(value_builder_->child(kInt32BitMaskIndex))), string_list_builder_( - reinterpret_cast(value_builder_.child(STRING_LIST_INDEX))), - int32_to_int32_list_builder_(reinterpret_cast( - value_builder_.child(INT32_TO_INT32_LIST_INDEX))) {} + reinterpret_cast(value_builder_->child(kStringListIndex))), + int32_to_int32_list_builder_( + reinterpret_cast(value_builder_->child(kInt32ToInt32ListIndex))) {} } // namespace internal } // namespace sql diff --git a/cpp/src/arrow/flight/sql/sql_info_internal.h b/cpp/src/arrow/flight/sql/sql_info_internal.h index 4f08b109303..cb222e6f880 100644 --- a/cpp/src/arrow/flight/sql/sql_info_internal.h +++ b/cpp/src/arrow/flight/sql/sql_info_internal.h @@ -28,40 +28,40 @@ namespace internal { /// data types. class SqlInfoResultAppender { public: - /// \brief Appends a string to the DenseUnionBuilder. + /// \brief Append a string to the DenseUnionBuilder. /// \param[in] value Value to be appended. Status operator()(const std::string& value); - /// \brief Appends a bool to the DenseUnionBuilder. + /// \brief Append a bool to the DenseUnionBuilder. /// \param[in] value Value to be appended. Status operator()(bool value); - /// \brief Appends a int64_t to the DenseUnionBuilder. + /// \brief Append a int64_t to the DenseUnionBuilder. /// \param[in] value Value to be appended. Status operator()(int64_t value); - /// \brief Appends a int64_t to the DenseUnionBuilder. + /// \brief Append a int32_t to the DenseUnionBuilder. /// \param[in] value Value to be appended. Status operator()(int32_t value); - /// \brief Appends a string list to the DenseUnionBuilder. + /// \brief Append a string list to the DenseUnionBuilder. /// \param[in] value Value to be appended. Status operator()(const std::vector& value); - /// \brief Appends a int32 to int32 list map to the DenseUnionBuilder. + /// \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 Creates a Variant visitor that appends data to given + /// \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); + explicit SqlInfoResultAppender(DenseUnionBuilder* value_builder); SqlInfoResultAppender(const SqlInfoResultAppender&) = delete; SqlInfoResultAppender(SqlInfoResultAppender&&) = delete; SqlInfoResultAppender& operator=(const SqlInfoResultAppender&) = delete; private: - DenseUnionBuilder& value_builder_; + DenseUnionBuilder* value_builder_; // Builders for each child on dense union StringBuilder* string_value_builder_; @@ -72,12 +72,12 @@ class SqlInfoResultAppender { MapBuilder* int32_to_int32_list_builder_; enum : int8_t { - STRING_VALUE_INDEX = 0, - BOOL_VALUE_INDEX = 1, - BIGINT_VALUE_INDEX = 2, - INT32_BITMASK_INDEX = 3, - STRING_LIST_INDEX = 4, - INT32_TO_INT32_LIST_INDEX = 5 + kStringValueIndex = 0, + kBoolValueIndex = 1, + kBigIntValueIndex = 2, + kInt32BitMaskIndex = 3, + kStringListIndex = 4, + kInt32ToInt32ListIndex = 5 }; }; diff --git a/cpp/src/arrow/flight/sql/test_app_cli.cc b/cpp/src/arrow/flight/sql/test_app_cli.cc index 5e3f9d7a660..2f3000d9131 100644 --- a/cpp/src/arrow/flight/sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/sql/test_app_cli.cc @@ -156,12 +156,9 @@ Status RunMain() { } else if (FLAGS_command == "GetTableTypes") { ARROW_ASSIGN_OR_RAISE(info, sql_client.GetTableTypes(call_options)); } else if (FLAGS_command == "GetTables") { - std::vector table_types = {}; - bool include_schema = false; - ARROW_ASSIGN_OR_RAISE( info, sql_client.GetTables(call_options, &FLAGS_catalog, &FLAGS_schema, - &FLAGS_table, include_schema, table_types)); + &FLAGS_table, false, nullptr)); } else if (FLAGS_command == "GetExportedKeys") { ARROW_ASSIGN_OR_RAISE(info, sql_client.GetExportedKeys(call_options, &FLAGS_catalog, &FLAGS_schema, FLAGS_table)); From 9e3c928cd870c81f1e2f4e2210c1fb3b05a4182e Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 25 Nov 2021 16:24:16 -0300 Subject: [PATCH 222/237] [C++] Address ratification comments (round 4 - part 2) (#213) * Make FlightSqlClient::GetFlightInfo return a Result instead of Status * Make most methods on FlightSqlServerBase to return a Result instead of Status * Move private functions on server.cc to anonymous namespace * Fixes on doxygen and better readability on server_test.cc * Rename fields on client_test.cc to follow the convention --- cpp/src/arrow/flight/sql/client.cc | 10 +- cpp/src/arrow/flight/sql/client.h | 15 +- cpp/src/arrow/flight/sql/client_test.cc | 126 ++++---- .../arrow/flight/sql/example/sqlite_server.cc | 195 ++++++------- .../arrow/flight/sql/example/sqlite_server.h | 115 ++++---- cpp/src/arrow/flight/sql/server.cc | 262 +++++++++-------- cpp/src/arrow/flight/sql/server.h | 275 ++++++++---------- cpp/src/arrow/flight/sql/server_test.cc | 90 +++++- 8 files changed, 551 insertions(+), 537 deletions(-) diff --git a/cpp/src/arrow/flight/sql/client.cc b/cpp/src/arrow/flight/sql/client.cc index d0ed9b4c6d7..833000e479d 100644 --- a/cpp/src/arrow/flight/sql/client.cc +++ b/cpp/src/arrow/flight/sql/client.cc @@ -71,9 +71,7 @@ arrow::Result> GetFlightInfoForCommand( const google::protobuf::Message& command) { const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command); - std::unique_ptr flight_info; - ARROW_RETURN_NOT_OK(client.GetFlightInfo(options, descriptor, &flight_info)); - + ARROW_ASSIGN_OR_RAISE(auto flight_info, client.GetFlightInfo(options, descriptor)); return std::move(flight_info); } @@ -340,10 +338,8 @@ arrow::Result> PreparedStatement::Execute() { ARROW_RETURN_NOT_OK(reader->ReadMetadata(&buffer)); } - std::unique_ptr info; - ARROW_RETURN_NOT_OK(client_->GetFlightInfo(options_, descriptor, &info)); - - return std::move(info); + ARROW_ASSIGN_OR_RAISE(auto flight_info, client_->GetFlightInfo(options_, descriptor)); + return std::move(flight_info); } arrow::Result PreparedStatement::ExecuteUpdate() { diff --git a/cpp/src/arrow/flight/sql/client.h b/cpp/src/arrow/flight/sql/client.h index b3986f1bf38..82ed35787e1 100644 --- a/cpp/src/arrow/flight/sql/client.h +++ b/cpp/src/arrow/flight/sql/client.h @@ -164,15 +164,16 @@ class ARROW_EXPORT FlightSqlClient { /// \brief Retrieve the FlightInfo. /// \param[in] options RPC-layer hints for this call. - /// \param[in] descriptor The flight descriptior. - /// \param[out] info The flight info with the metadata. - /// \return Status. + /// \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 Status GetFlightInfo(const FlightCallOptions& options, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { - return impl_->GetFlightInfo(options, descriptor, info); + 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: diff --git a/cpp/src/arrow/flight/sql/client_test.cc b/cpp/src/arrow/flight/sql/client_test.cc index 4bb3e859a91..b2b08c78ee7 100644 --- a/cpp/src/arrow/flight/sql/client_test.cc +++ b/cpp/src/arrow/flight/sql/client_test.cc @@ -41,9 +41,8 @@ class FlightSqlClientMock : public FlightSqlClient { ~FlightSqlClientMock() = default; - MOCK_METHOD(Status, GetFlightInfo, - (const FlightCallOptions&, const FlightDescriptor&, - std::unique_ptr*)); + MOCK_METHOD(arrow::Result>, GetFlightInfo, + (const FlightCallOptions&, const FlightDescriptor&)); MOCK_METHOD(Status, DoGet, (const FlightCallOptions& options, const Ticket& ticket, std::unique_ptr* stream)); @@ -59,8 +58,8 @@ class FlightSqlClientMock : public FlightSqlClient { class TestFlightSqlClient : public ::testing::Test { protected: - FlightSqlClientMock sql_client; - FlightCallOptions call_options; + FlightSqlClientMock sql_client_; + FlightCallOptions call_options_; void SetUp() override {} @@ -120,13 +119,20 @@ FlightDescriptor getDescriptor(google::protobuf::Message& command) { 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); - EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); - ASSERT_OK(sql_client.GetCatalogs(call_options)); + ASSERT_OK(sql_client_.GetCatalogs(call_options_)); } TEST_F(TestFlightSqlClient, TestGetSchemas) { @@ -138,9 +144,10 @@ TEST_F(TestFlightSqlClient, TestGetSchemas) { command.set_schema_filter_pattern(schema_filter_pattern); FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); - ASSERT_OK(sql_client.GetSchemas(call_options, &catalog, &schema_filter_pattern)); + ASSERT_OK(sql_client_.GetSchemas(call_options_, &catalog, &schema_filter_pattern)); } TEST_F(TestFlightSqlClient, TestGetTables) { @@ -160,20 +167,22 @@ TEST_F(TestFlightSqlClient, TestGetTables) { } FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); + 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)); + 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); - EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); - ASSERT_OK(sql_client.GetTableTypes(call_options)); + ASSERT_OK(sql_client_.GetTableTypes(call_options_)); } TEST_F(TestFlightSqlClient, TestGetExported) { @@ -187,9 +196,10 @@ TEST_F(TestFlightSqlClient, TestGetExported) { command.set_table(table); FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); - ASSERT_OK(sql_client.GetExportedKeys(call_options, &catalog, &schema, table)); + ASSERT_OK(sql_client_.GetExportedKeys(call_options_, &catalog, &schema, table)); } TEST_F(TestFlightSqlClient, TestGetImported) { @@ -203,9 +213,10 @@ TEST_F(TestFlightSqlClient, TestGetImported) { command.set_table(table); FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); - ASSERT_OK(sql_client.GetImportedKeys(call_options, &catalog, &schema, table)); + ASSERT_OK(sql_client_.GetImportedKeys(call_options_, &catalog, &schema, table)); } TEST_F(TestFlightSqlClient, TestGetPrimary) { @@ -219,9 +230,10 @@ TEST_F(TestFlightSqlClient, TestGetPrimary) { command.set_table(table); FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); - ASSERT_OK(sql_client.GetPrimaryKeys(call_options, &catalog, &schema, table)); + ASSERT_OK(sql_client_.GetPrimaryKeys(call_options_, &catalog, &schema, table)); } TEST_F(TestFlightSqlClient, TestGetCrossReference) { @@ -241,10 +253,11 @@ TEST_F(TestFlightSqlClient, TestGetCrossReference) { command.set_fk_table(fk_table); FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); - ASSERT_OK(sql_client.GetCrossReference(call_options, &pk_catalog, &pk_schema, pk_table, - &fk_catalog, &fk_schema, fk_table)); + ASSERT_OK(sql_client_.GetCrossReference(call_options_, &pk_catalog, &pk_schema, + pk_table, &fk_catalog, &fk_schema, fk_table)); } TEST_F(TestFlightSqlClient, TestExecute) { @@ -254,15 +267,16 @@ TEST_F(TestFlightSqlClient, TestExecute) { command.set_query(query); FlightDescriptor descriptor = getDescriptor(command); - EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(Ref(call_options_), descriptor)); - ASSERT_OK(sql_client.Execute(call_options, query)); + ASSERT_OK(sql_client_.Execute(call_options_, query)); } TEST_F(TestFlightSqlClient, TestPreparedStatementExecute) { const std::string query = "query"; - ON_CALL(sql_client, DoAction) + ON_CALL(sql_client_, DoAction) .WillByDefault([](const FlightCallOptions& options, const Action& action, std::unique_ptr* results) { google::protobuf::Any command; @@ -279,11 +293,13 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecute) { return Status::OK(); }); - EXPECT_CALL(sql_client, DoAction(_, _, _)).Times(2); + EXPECT_CALL(sql_client_, DoAction(_, _, _)).Times(2); - ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client.Prepare(call_options, query)); + ASSERT_OK_AND_ASSIGN(auto prepared_statement, + sql_client_.Prepare(call_options_, query)); - EXPECT_CALL(sql_client, GetFlightInfo(_, _, _)); + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(_, _)); ASSERT_OK(prepared_statement->Execute()); } @@ -291,7 +307,7 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecute) { TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { const std::string query = "query"; - ON_CALL(sql_client, DoAction) + ON_CALL(sql_client_, DoAction) .WillByDefault([](const FlightCallOptions& options, const Action& action, std::unique_ptr* results) { google::protobuf::Any command; @@ -319,7 +335,7 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { }); std::shared_ptr buffer_ptr; - ON_CALL(sql_client, DoPut) + ON_CALL(sql_client_, DoPut) .WillByDefault([&buffer_ptr](const FlightCallOptions& options, const FlightDescriptor& descriptor1, const std::shared_ptr& schema, @@ -331,17 +347,19 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteParameterBinding) { return Status::OK(); }); - EXPECT_CALL(sql_client, DoAction(_, _, _)).Times(2); - EXPECT_CALL(sql_client, DoPut(_, _, _, _, _)); + 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)); + 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)); - EXPECT_CALL(sql_client, GetFlightInfo(_, _, _)); + ON_CALL(sql_client_, GetFlightInfo).WillByDefault(ReturnEmptyFlightInfo); + EXPECT_CALL(sql_client_, GetFlightInfo(_, _)); ASSERT_OK(prepared_statement->Execute()); } @@ -365,7 +383,7 @@ TEST_F(TestFlightSqlClient, TestExecuteUpdate) { auto buffer_ptr = std::make_shared( reinterpret_cast(string.data()), doPutUpdateResult.ByteSizeLong()); - ON_CALL(sql_client, DoPut) + ON_CALL(sql_client_, DoPut) .WillByDefault([&buffer_ptr](const FlightCallOptions& options, const FlightDescriptor& descriptor1, const std::shared_ptr& schema, @@ -379,9 +397,9 @@ TEST_F(TestFlightSqlClient, TestExecuteUpdate) { std::unique_ptr flight_info; std::unique_ptr writer; std::unique_ptr reader; - EXPECT_CALL(sql_client, DoPut(Ref(call_options), descriptor, _, _, _)); + EXPECT_CALL(sql_client_, DoPut(Ref(call_options_), descriptor, _, _, _)); - ASSERT_OK_AND_ASSIGN(auto num_rows, sql_client.ExecuteUpdate(call_options, query)); + ASSERT_OK_AND_ASSIGN(auto num_rows, sql_client_.ExecuteUpdate(call_options_, query)); ASSERT_EQ(num_rows, 100); } @@ -397,19 +415,21 @@ TEST_F(TestFlightSqlClient, TestGetSqlInfo) { any.PackFrom(command); const FlightDescriptor& descriptor = FlightDescriptor::Command(any.SerializeAsString()); - EXPECT_CALL(sql_client, GetFlightInfo(Ref(call_options), descriptor, _)); - ASSERT_OK(sql_client.GetSqlInfo(call_options, sql_info)); + 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) { + 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) + ON_CALL(sql_client_, DoAction) .WillByDefault([&query, &schema](const FlightCallOptions& options, const Action& action, std::unique_ptr* results) { @@ -433,10 +453,10 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( return Status::OK(); }); - EXPECT_CALL(sql_client, DoAction(_, _, _)).Times(2); + EXPECT_CALL(sql_client_, DoAction(_, _, _)).Times(2); auto buffer = Buffer::FromString(result.SerializeAsString()); - ON_CALL(sql_client, DoPut) + ON_CALL(sql_client_, DoPut) .WillByDefault([&buffer](const FlightCallOptions& options, const FlightDescriptor& descriptor1, const std::shared_ptr& schema, @@ -447,13 +467,13 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( return Status::OK(); }); if (schema == NULLPTR) { - EXPECT_CALL(sql_client, DoPut(_, _, _, _, _)); + EXPECT_CALL(sql_client_, DoPut(_, _, _, _, _)); } else { - EXPECT_CALL(sql_client, DoPut(_, _, *schema, _, _)); + 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 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()); @@ -462,9 +482,9 @@ inline void AssertTestPreparedStatementExecuteUpdateOk( TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateNoParameterBinding) { AssertTestPreparedStatementExecuteUpdateOk( [](const std::shared_ptr& prepared_statement, - FlightSqlClient& sql_client, const std::shared_ptr* schema, + FlightSqlClient& sql_client_, const std::shared_ptr* schema, const int64_t& row_count) {}, - NULLPTR, sql_client); + NULLPTR, sql_client_); } TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBinding) { @@ -472,7 +492,7 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBindi {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, + FlightSqlClient& sql_client_, const std::shared_ptr* schema, const int64_t& row_count) { auto string_array = ArrayFromJSON(utf8(), R"(["Lorem", "Ipsum", "Foo", "Bar", "Baz"])"); @@ -481,7 +501,7 @@ TEST_F(TestFlightSqlClient, TestPreparedStatementExecuteUpdateWithParameterBindi RecordBatch::Make(*schema, row_count, {string_array, uint8_array}); ASSERT_OK(prepared_statement->SetParameters(recordBatch)); }, - &schema, sql_client); + &schema, sql_client_); } } // namespace sql diff --git a/cpp/src/arrow/flight/sql/example/sqlite_server.cc b/cpp/src/arrow/flight/sql/example/sqlite_server.cc index 8854b9c5311..cac80163d71 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.cc @@ -225,9 +225,8 @@ Status SQLiteFlightSqlServer::ExecuteSql(const std::string& sql) { return Status::OK(); } -Status DoGetSQLiteQuery(sqlite3* db, const std::string& query, - const std::shared_ptr& schema, - std::unique_ptr* result) { +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)); @@ -235,27 +234,21 @@ Status DoGetSQLiteQuery(sqlite3* db, const std::string& query, std::shared_ptr reader; ARROW_ASSIGN_OR_RAISE(reader, SqliteStatementBatchReader::Create(statement, schema)); - *result = std::unique_ptr(new RecordBatchStream(reader)); - - return Status::OK(); + return std::unique_ptr(new RecordBatchStream(reader)); } -Status GetFlightInfoForCommand(const FlightDescriptor& descriptor, - std::unique_ptr* info, - const std::shared_ptr& schema) { +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)) - *info = std::unique_ptr(new FlightInfo(result)); - - return Status::OK(); + return std::unique_ptr(new FlightInfo(result)); } -Status SQLiteFlightSqlServer::GetFlightInfoStatement(const ServerCallContext& context, - const StatementQuery& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { +arrow::Result> SQLiteFlightSqlServer::GetFlightInfoStatement( + const ServerCallContext& context, const StatementQuery& command, + const FlightDescriptor& descriptor) { const std::string& query = command.query; std::shared_ptr statement; @@ -270,14 +263,11 @@ Status SQLiteFlightSqlServer::GetFlightInfoStatement(const ServerCallContext& co ARROW_ASSIGN_OR_RAISE(auto result, FlightInfo::Make(*schema, descriptor, endpoints, -1, -1)) - *info = std::unique_ptr(new FlightInfo(result)); - - return Status::OK(); + return std::unique_ptr(new FlightInfo(result)); } -Status SQLiteFlightSqlServer::DoGetStatement(const ServerCallContext& context, - const StatementQueryTicket& command, - std::unique_ptr* result) { +arrow::Result> SQLiteFlightSqlServer::DoGetStatement( + const ServerCallContext& context, const StatementQueryTicket& command) { const std::string& sql = command.statement_handle; std::shared_ptr statement; @@ -286,19 +276,16 @@ Status SQLiteFlightSqlServer::DoGetStatement(const ServerCallContext& context, std::shared_ptr reader; ARROW_ASSIGN_OR_RAISE(reader, SqliteStatementBatchReader::Create(statement)); - *result = std::unique_ptr(new RecordBatchStream(reader)); - - return Status::OK(); + return std::unique_ptr(new RecordBatchStream(reader)); } -Status SQLiteFlightSqlServer::GetFlightInfoCatalogs(const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetCatalogsSchema()); +arrow::Result> SQLiteFlightSqlServer::GetFlightInfoCatalogs( + const ServerCallContext& context, const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetCatalogsSchema()); } -Status SQLiteFlightSqlServer::DoGetCatalogs(const ServerCallContext& context, - std::unique_ptr* result) { +arrow::Result> SQLiteFlightSqlServer::DoGetCatalogs( + const ServerCallContext& context) { // As SQLite doesn't support catalogs, this will return an empty record batch. const std::shared_ptr& schema = SqlSchema::GetCatalogsSchema(); @@ -310,20 +297,18 @@ Status SQLiteFlightSqlServer::DoGetCatalogs(const ServerCallContext& context, RecordBatch::Make(schema, 0, {catalog_name}); ARROW_ASSIGN_OR_RAISE(auto reader, RecordBatchReader::Make({batch})); - *result = std::unique_ptr(new RecordBatchStream(reader)); - return Status::OK(); + + return std::unique_ptr(new RecordBatchStream(reader)); } -Status SQLiteFlightSqlServer::GetFlightInfoSchemas(const ServerCallContext& context, - const GetSchemas& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetSchemasSchema()); +arrow::Result> SQLiteFlightSqlServer::GetFlightInfoSchemas( + const ServerCallContext& context, const GetSchemas& command, + const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetSchemasSchema()); } -Status SQLiteFlightSqlServer::DoGetSchemas(const ServerCallContext& context, - const GetSchemas& command, - std::unique_ptr* result) { +arrow::Result> SQLiteFlightSqlServer::DoGetSchemas( + const ServerCallContext& context, const GetSchemas& command) { // As SQLite doesn't support schemas, this will return an empty record batch. const std::shared_ptr& schema = SqlSchema::GetSchemasSchema(); @@ -337,14 +322,13 @@ Status SQLiteFlightSqlServer::DoGetSchemas(const ServerCallContext& context, RecordBatch::Make(schema, 0, {catalog_name, schema_name}); ARROW_ASSIGN_OR_RAISE(auto reader, RecordBatchReader::Make({batch})); - *result = std::unique_ptr(new RecordBatchStream(reader)); - return Status::OK(); + + return std::unique_ptr(new RecordBatchStream(reader)); } -Status SQLiteFlightSqlServer::GetFlightInfoTables(const ServerCallContext& context, - const GetTables& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { +arrow::Result> SQLiteFlightSqlServer::GetFlightInfoTables( + const ServerCallContext& context, const GetTables& command, + const FlightDescriptor& descriptor) { std::vector endpoints{FlightEndpoint{{descriptor.cmd}, {}}}; bool include_schema = command.include_schema; @@ -354,14 +338,12 @@ Status SQLiteFlightSqlServer::GetFlightInfoTables(const ServerCallContext& conte FlightInfo::Make(include_schema ? *SqlSchema::GetTablesSchemaWithIncludedSchema() : *SqlSchema::GetTablesSchema(), descriptor, endpoints, -1, -1)) - *info = std::unique_ptr(new FlightInfo(result)); - return Status::OK(); + return std::unique_ptr(new FlightInfo(result)); } -Status SQLiteFlightSqlServer::DoGetTables(const ServerCallContext& context, - const GetTables& command, - std::unique_ptr* result) { +arrow::Result> SQLiteFlightSqlServer::DoGetTables( + const ServerCallContext& context, const GetTables& command) { std::string query = PrepareQueryForGetTables(command); std::shared_ptr statement; @@ -374,13 +356,10 @@ Status SQLiteFlightSqlServer::DoGetTables(const ServerCallContext& context, if (command.include_schema) { std::shared_ptr table_schema_reader = std::make_shared(reader, query, db_); - *result = - std::unique_ptr(new RecordBatchStream(table_schema_reader)); + return std::unique_ptr(new RecordBatchStream(table_schema_reader)); } else { - *result = std::unique_ptr(new RecordBatchStream(reader)); + return std::unique_ptr(new RecordBatchStream(reader)); } - - return Status::OK(); } arrow::Result SQLiteFlightSqlServer::DoPutCommandStatementUpdate( @@ -441,8 +420,8 @@ SQLiteFlightSqlServer::CreatePreparedStatement( } Status SQLiteFlightSqlServer::ClosePreparedStatement( - const ServerCallContext& context, const ActionClosePreparedStatementRequest& request, - std::unique_ptr* result) { + const ServerCallContext& context, + const ActionClosePreparedStatementRequest& request) { const std::string& prepared_statement_handle = request.prepared_statement_handle; const auto& uuid = boost::lexical_cast(prepared_statement_handle); @@ -453,14 +432,13 @@ Status SQLiteFlightSqlServer::ClosePreparedStatement( return Status::Invalid("Prepared statement not found"); } - // Need to instantiate a ResultStream, otherwise clients can not wait for completion. - *result = std::unique_ptr(new SimpleResultStream({})); return Status::OK(); } -Status SQLiteFlightSqlServer::GetFlightInfoPreparedStatement( +arrow::Result> +SQLiteFlightSqlServer::GetFlightInfoPreparedStatement( const ServerCallContext& context, const PreparedStatementQuery& command, - const FlightDescriptor& descriptor, std::unique_ptr* info) { + const FlightDescriptor& descriptor) { const std::string& prepared_statement_handle = command.prepared_statement_handle; const auto& uuid = boost::lexical_cast(prepared_statement_handle); @@ -474,12 +452,12 @@ Status SQLiteFlightSqlServer::GetFlightInfoPreparedStatement( std::shared_ptr schema; ARROW_RETURN_NOT_OK(statement->GetSchema(&schema)); - return GetFlightInfoForCommand(descriptor, info, schema); + return GetFlightInfoForCommand(descriptor, schema); } -Status SQLiteFlightSqlServer::DoGetPreparedStatement( - const ServerCallContext& context, const PreparedStatementQuery& command, - std::unique_ptr* result) { +arrow::Result> +SQLiteFlightSqlServer::DoGetPreparedStatement(const ServerCallContext& context, + const PreparedStatementQuery& command) { const std::string& prepared_statement_handle = command.prepared_statement_handle; const auto& uuid = boost::lexical_cast(prepared_statement_handle); @@ -493,9 +471,7 @@ Status SQLiteFlightSqlServer::DoGetPreparedStatement( std::shared_ptr reader; ARROW_ASSIGN_OR_RAISE(reader, SqliteStatementBatchReader::Create(statement)); - *result = std::unique_ptr(new RecordBatchStream(reader)); - - return Status::OK(); + return std::unique_ptr(new RecordBatchStream(reader)); } Status SQLiteFlightSqlServer::DoPutPreparedStatementQuery( @@ -527,28 +503,27 @@ arrow::Result SQLiteFlightSqlServer::DoPutPreparedStatementUpdate( return record_count; } -Status SQLiteFlightSqlServer::GetFlightInfoTableTypes(const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetTableTypesSchema()); +arrow::Result> SQLiteFlightSqlServer::GetFlightInfoTableTypes( + const ServerCallContext& context, const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetTableTypesSchema()); } -Status SQLiteFlightSqlServer::DoGetTableTypes(const ServerCallContext& context, - std::unique_ptr* result) { +arrow::Result> SQLiteFlightSqlServer::DoGetTableTypes( + const ServerCallContext& context) { std::string query = "SELECT DISTINCT type as table_type FROM sqlite_master"; - return DoGetSQLiteQuery(db_, query, SqlSchema::GetTableTypesSchema(), result); + return DoGetSQLiteQuery(db_, query, SqlSchema::GetTableTypesSchema()); } -Status SQLiteFlightSqlServer::GetFlightInfoPrimaryKeys( - const ServerCallContext& context, const GetPrimaryKeys& command, - const FlightDescriptor& descriptor, std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetPrimaryKeysSchema()); +arrow::Result> +SQLiteFlightSqlServer::GetFlightInfoPrimaryKeys(const ServerCallContext& context, + const GetPrimaryKeys& command, + const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetPrimaryKeysSchema()); } -Status SQLiteFlightSqlServer::DoGetPrimaryKeys( - const ServerCallContext& context, const GetPrimaryKeys& command, - std::unique_ptr* result) { +arrow::Result> SQLiteFlightSqlServer::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 @@ -570,8 +545,7 @@ Status SQLiteFlightSqlServer::DoGetPrimaryKeys( table_query << " and table_name LIKE '" << command.table << "'"; - return DoGetSQLiteQuery(db_, table_query.str(), SqlSchema::GetPrimaryKeysSchema(), - result); + return DoGetSQLiteQuery(db_, table_query.str(), SqlSchema::GetPrimaryKeysSchema()); } std::string PrepareQueryForGetImportedOrExportedKeys(const std::string& filter) { @@ -607,15 +581,15 @@ std::string PrepareQueryForGetImportedOrExportedKeys(const std::string& filter) pk_catalog_name, pk_schema_name, pk_table_name, pk_key_name, key_sequence)"; } -Status SQLiteFlightSqlServer::GetFlightInfoImportedKeys( - const ServerCallContext& context, const GetImportedKeys& command, - const FlightDescriptor& descriptor, std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetImportedKeysSchema()); +arrow::Result> +SQLiteFlightSqlServer::GetFlightInfoImportedKeys(const ServerCallContext& context, + const GetImportedKeys& command, + const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetImportedKeysSchema()); } -Status SQLiteFlightSqlServer::DoGetImportedKeys( - const ServerCallContext& context, const GetImportedKeys& command, - std::unique_ptr* result) { +arrow::Result> SQLiteFlightSqlServer::DoGetImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command) { std::string filter = "fk_table_name = '" + command.table + "'"; if (command.catalog.has_value()) { filter += " AND fk_catalog_name = '" + command.catalog.value() + "'"; @@ -625,18 +599,18 @@ Status SQLiteFlightSqlServer::DoGetImportedKeys( } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); - return DoGetSQLiteQuery(db_, query, SqlSchema::GetImportedKeysSchema(), result); + return DoGetSQLiteQuery(db_, query, SqlSchema::GetImportedKeysSchema()); } -Status SQLiteFlightSqlServer::GetFlightInfoExportedKeys( - const ServerCallContext& context, const GetExportedKeys& command, - const FlightDescriptor& descriptor, std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetExportedKeysSchema()); +arrow::Result> +SQLiteFlightSqlServer::GetFlightInfoExportedKeys(const ServerCallContext& context, + const GetExportedKeys& command, + const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetExportedKeysSchema()); } -Status SQLiteFlightSqlServer::DoGetExportedKeys( - const ServerCallContext& context, const GetExportedKeys& command, - std::unique_ptr* result) { +arrow::Result> SQLiteFlightSqlServer::DoGetExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command) { std::string filter = "pk_table_name = '" + command.table + "'"; if (command.catalog.has_value()) { filter += " AND pk_catalog_name = '" + command.catalog.value() + "'"; @@ -646,18 +620,19 @@ Status SQLiteFlightSqlServer::DoGetExportedKeys( } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); - return DoGetSQLiteQuery(db_, query, SqlSchema::GetExportedKeysSchema(), result); + return DoGetSQLiteQuery(db_, query, SqlSchema::GetExportedKeysSchema()); } -Status SQLiteFlightSqlServer::GetFlightInfoCrossReference( - const ServerCallContext& context, const GetCrossReference& command, - const FlightDescriptor& descriptor, std::unique_ptr* info) { - return GetFlightInfoForCommand(descriptor, info, SqlSchema::GetCrossReferenceSchema()); +arrow::Result> +SQLiteFlightSqlServer::GetFlightInfoCrossReference(const ServerCallContext& context, + const GetCrossReference& command, + const FlightDescriptor& descriptor) { + return GetFlightInfoForCommand(descriptor, SqlSchema::GetCrossReferenceSchema()); } -Status SQLiteFlightSqlServer::DoGetCrossReference( - const ServerCallContext& context, const GetCrossReference& command, - std::unique_ptr* result) { +arrow::Result> +SQLiteFlightSqlServer::DoGetCrossReference(const ServerCallContext& context, + const GetCrossReference& command) { std::string filter = "pk_table_name = '" + command.pk_table + "'"; if (command.pk_catalog.has_value()) { filter += " AND pk_catalog_name = '" + command.pk_catalog.value() + "'"; @@ -675,7 +650,7 @@ Status SQLiteFlightSqlServer::DoGetCrossReference( } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); - return DoGetSQLiteQuery(db_, query, SqlSchema::GetCrossReferenceSchema(), result); + return DoGetSQLiteQuery(db_, query, SqlSchema::GetCrossReferenceSchema()); } } // namespace example diff --git a/cpp/src/arrow/flight/sql/example/sqlite_server.h b/cpp/src/arrow/flight/sql/example/sqlite_server.h index 6eb890d5ddf..58f88d9d5d3 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.h @@ -258,40 +258,35 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { /// SQLite database. Status ExecuteSql(const std::string& sql); - Status GetFlightInfoStatement(const ServerCallContext& context, - const StatementQuery& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) override; + arrow::Result> GetFlightInfoStatement( + const ServerCallContext& context, const StatementQuery& command, + const FlightDescriptor& descriptor) override; - Status DoGetStatement(const ServerCallContext& context, - const StatementQueryTicket& command, - std::unique_ptr* result) override; - Status GetFlightInfoCatalogs(const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info) override; - Status DoGetCatalogs(const ServerCallContext& context, - std::unique_ptr* result) override; - Status GetFlightInfoSchemas(const ServerCallContext& context, const GetSchemas& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) override; - Status DoGetSchemas(const ServerCallContext& context, const GetSchemas& command, - std::unique_ptr* result) 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 GetSchemas& command, + const FlightDescriptor& descriptor) override; + arrow::Result> DoGetSchemas( + const ServerCallContext& context, const GetSchemas& command) override; arrow::Result DoPutCommandStatementUpdate( const ServerCallContext& context, const StatementUpdate& update, std::unique_ptr& reader) override; arrow::Result CreatePreparedStatement( const ServerCallContext& context, const ActionCreatePreparedStatementRequest& request) override; - Status ClosePreparedStatement(const ServerCallContext& context, - const ActionClosePreparedStatementRequest& request, - std::unique_ptr* result) override; - Status GetFlightInfoPreparedStatement(const ServerCallContext& context, - const PreparedStatementQuery& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) override; - Status DoGetPreparedStatement(const ServerCallContext& context, - const PreparedStatementQuery& command, - std::unique_ptr* result) 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, @@ -300,46 +295,38 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { const ServerCallContext& context, const PreparedStatementUpdate& command, FlightMessageReader* reader) override; - Status GetFlightInfoTables(const ServerCallContext& context, const GetTables& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) override; + arrow::Result> GetFlightInfoTables( + const ServerCallContext& context, const GetTables& command, + const FlightDescriptor& descriptor) override; - Status DoGetTables(const ServerCallContext& context, const GetTables& command, - std::unique_ptr* result) override; - Status GetFlightInfoTableTypes(const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info) override; - Status DoGetTableTypes(const ServerCallContext& context, - std::unique_ptr* result) override; - Status GetFlightInfoImportedKeys(const ServerCallContext& context, - const GetImportedKeys& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) override; - Status DoGetImportedKeys(const ServerCallContext& context, - const GetImportedKeys& command, - std::unique_ptr* result) override; - Status GetFlightInfoExportedKeys(const ServerCallContext& context, - const GetExportedKeys& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) override; - Status DoGetExportedKeys(const ServerCallContext& context, - const GetExportedKeys& command, - std::unique_ptr* result) override; - Status GetFlightInfoCrossReference(const ServerCallContext& context, - const GetCrossReference& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) override; - Status DoGetCrossReference(const ServerCallContext& context, - const GetCrossReference& command, - std::unique_ptr* result) 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; - Status GetFlightInfoPrimaryKeys(const ServerCallContext& context, - const GetPrimaryKeys& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) override; + arrow::Result> GetFlightInfoPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command, + const FlightDescriptor& descriptor) override; - Status DoGetPrimaryKeys(const ServerCallContext& context, const GetPrimaryKeys& command, - std::unique_ptr* result) override; + arrow::Result> DoGetPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command) override; private: sqlite3* db_; diff --git a/cpp/src/arrow/flight/sql/server.cc b/cpp/src/arrow/flight/sql/server.cc index f3f83bc8a5f..219ffd8e01b 100644 --- a/cpp/src/arrow/flight/sql/server.cc +++ b/cpp/src/arrow/flight/sql/server.cc @@ -28,7 +28,6 @@ #include "arrow/flight/sql/sql_info_internal.h" #include "arrow/type.h" #include "arrow/util/checked_cast.h" -#include "arrow/util/logging.h" #define PROPERTY_TO_OPTIONAL(COMMAND, PROPERTY) \ COMMAND.has_##PROPERTY() ? util::make_optional(COMMAND.PROPERTY()) : util::nullopt; @@ -42,21 +41,7 @@ namespace pb = arrow::flight::protocol; using arrow::internal::checked_cast; using arrow::internal::checked_pointer_cast; -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; -} +namespace { arrow::Result ParseCommandGetCrossReference( const google::protobuf::Any& any) { @@ -253,6 +238,24 @@ ParseActionClosePreparedStatementRequest(const google::protobuf::Any& any) { 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) { @@ -264,41 +267,60 @@ Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, if (any.Is()) { ARROW_ASSIGN_OR_RAISE(StatementQuery internal_command, ParseCommandStatementQuery(any)); - return GetFlightInfoStatement(context, internal_command, request, info); + 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)); - return GetFlightInfoPreparedStatement(context, internal_command, request, info); + ARROW_ASSIGN_OR_RAISE( + *info, GetFlightInfoPreparedStatement(context, internal_command, request)); + return Status::OK(); } else if (any.Is()) { - return GetFlightInfoCatalogs(context, request, info); + ARROW_ASSIGN_OR_RAISE(*info, GetFlightInfoCatalogs(context, request)); + return Status::OK(); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetSchemas internal_command, ParseCommandGetSchemas(any)); - return GetFlightInfoSchemas(context, internal_command, request, info); + 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)); - return GetFlightInfoTables(context, command, request, info); + ARROW_ASSIGN_OR_RAISE(*info, GetFlightInfoTables(context, command, request)); + return Status::OK(); } else if (any.Is()) { - return GetFlightInfoTableTypes(context, request, info); + 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_)); - return GetFlightInfoSqlInfo(context, internal_command, request, info); + 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)); - return GetFlightInfoPrimaryKeys(context, internal_command, request, info); + 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)); - return GetFlightInfoExportedKeys(context, internal_command, request, info); + 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)); - return GetFlightInfoImportedKeys(context, internal_command, request, info); + 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)); - return GetFlightInfoCrossReference(context, internal_command, request, info); + ARROW_ASSIGN_OR_RAISE( + *info, GetFlightInfoCrossReference(context, internal_command, request)); + return Status::OK(); } return Status::Invalid("The defined request is invalid."); @@ -315,41 +337,52 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket if (any.Is()) { ARROW_ASSIGN_OR_RAISE(StatementQueryTicket command, ParseStatementQueryTicket(any)); - return DoGetStatement(context, command, stream); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetStatement(context, command)); + return Status::OK(); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command, ParseCommandPreparedStatementQuery(any)); - return DoGetPreparedStatement(context, internal_command, stream); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetPreparedStatement(context, internal_command)); + return Status::OK(); } else if (any.Is()) { - return DoGetCatalogs(context, stream); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetCatalogs(context)); + return Status::OK(); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetSchemas internal_command, ParseCommandGetSchemas(any)); - return DoGetSchemas(context, internal_command, stream); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetSchemas(context, internal_command)); + return Status::OK(); } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(GetTables command, ParseCommandGetTables(any)); - return DoGetTables(context, command, stream); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetTables(context, command)); + return Status::OK(); } else if (any.Is()) { - return DoGetTableTypes(context, stream); + 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_)); - return DoGetSqlInfo(context, internal_command, stream); + 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)); - return DoGetPrimaryKeys(context, internal_command, stream); + 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)); - return DoGetExportedKeys(context, internal_command, stream); + 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)); - return DoGetImportedKeys(context, internal_command, stream); + 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)); - return DoGetCrossReference(context, internal_command, stream); + ARROW_ASSIGN_OR_RAISE(*stream, DoGetCrossReference(context, internal_command)); + return Status::OK(); } return Status::Invalid("The defined request is invalid."); @@ -451,75 +484,69 @@ Status FlightSqlServerBase::DoAction(const ServerCallContext& context, ARROW_ASSIGN_OR_RAISE(ActionClosePreparedStatementRequest internal_command, ParseActionClosePreparedStatementRequest(any)); - return ClosePreparedStatement(context, internal_command, result_stream); + 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."); } -Status FlightSqlServerBase::GetFlightInfoCatalogs(const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { +arrow::Result> FlightSqlServerBase::GetFlightInfoCatalogs( + const ServerCallContext& context, const FlightDescriptor& descriptor) { return Status::NotImplemented("GetFlightInfoCatalogs not implemented"); } -Status FlightSqlServerBase::DoGetCatalogs(const ServerCallContext& context, - std::unique_ptr* result) { +arrow::Result> FlightSqlServerBase::DoGetCatalogs( + const ServerCallContext& context) { return Status::NotImplemented("DoGetCatalogs not implemented"); } -Status FlightSqlServerBase::GetFlightInfoStatement(const ServerCallContext& context, - const StatementQuery& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { +arrow::Result> FlightSqlServerBase::GetFlightInfoStatement( + const ServerCallContext& context, const StatementQuery& command, + const FlightDescriptor& descriptor) { return Status::NotImplemented("GetFlightInfoStatement not implemented"); } -Status FlightSqlServerBase::DoGetStatement(const ServerCallContext& context, - const StatementQueryTicket& command, - std::unique_ptr* result) { +arrow::Result> FlightSqlServerBase::DoGetStatement( + const ServerCallContext& context, const StatementQueryTicket& command) { return Status::NotImplemented("DoGetStatement not implemented"); } -Status FlightSqlServerBase::GetFlightInfoPreparedStatement( - const ServerCallContext& context, const PreparedStatementQuery& command, - const FlightDescriptor& descriptor, std::unique_ptr* info) { +arrow::Result> +FlightSqlServerBase::GetFlightInfoPreparedStatement(const ServerCallContext& context, + const PreparedStatementQuery& command, + const FlightDescriptor& descriptor) { return Status::NotImplemented("GetFlightInfoPreparedStatement not implemented"); } -Status FlightSqlServerBase::DoGetPreparedStatement( - const ServerCallContext& context, const PreparedStatementQuery& command, - std::unique_ptr* result) { +arrow::Result> +FlightSqlServerBase::DoGetPreparedStatement(const ServerCallContext& context, + const PreparedStatementQuery& command) { return Status::NotImplemented("DoGetPreparedStatement not implemented"); } -Status FlightSqlServerBase::GetFlightInfoSqlInfo(const ServerCallContext& context, - const GetSqlInfo& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { - DCHECK(info); - +arrow::Result> FlightSqlServerBase::GetFlightInfoSqlInfo( + const ServerCallContext& context, const GetSqlInfo& command, + const FlightDescriptor& descriptor) { if (sql_info_id_to_result_.empty()) { - return Status::NotImplemented("GetFlightInfoSqlInfo not implemented"); + 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)) - *info = std::unique_ptr(new FlightInfo(result)); - - return Status::OK(); + return std::unique_ptr(new FlightInfo(result)); } void FlightSqlServerBase::RegisterSqlInfo(int32_t id, const SqlInfoResult& result) { sql_info_id_to_result_[id] = result; } -Status FlightSqlServerBase::DoGetSqlInfo(const ServerCallContext& context, - const GetSqlInfo& command, - std::unique_ptr* result) { - DCHECK(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; @@ -553,95 +580,84 @@ Status FlightSqlServerBase::DoGetSqlInfo(const ServerCallContext& context, const std::shared_ptr& batch = RecordBatch::Make(SqlSchema::GetSqlInfoSchema(), row_count, {name, value}); ARROW_ASSIGN_OR_RAISE(const auto reader, RecordBatchReader::Make({batch})); - *result = std::unique_ptr(new RecordBatchStream(reader)); - return Status::OK(); + + return std::unique_ptr(new RecordBatchStream(reader)); } -Status FlightSqlServerBase::GetFlightInfoSchemas(const ServerCallContext& context, - const GetSchemas& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { +arrow::Result> FlightSqlServerBase::GetFlightInfoSchemas( + const ServerCallContext& context, const GetSchemas& command, + const FlightDescriptor& descriptor) { return Status::NotImplemented("GetFlightInfoSchemas not implemented"); } -Status FlightSqlServerBase::DoGetSchemas(const ServerCallContext& context, - const GetSchemas& command, - std::unique_ptr* result) { +arrow::Result> FlightSqlServerBase::DoGetSchemas( + const ServerCallContext& context, const GetSchemas& command) { return Status::NotImplemented("DoGetSchemas not implemented"); } -Status FlightSqlServerBase::GetFlightInfoTables(const ServerCallContext& context, - const GetTables& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { +arrow::Result> FlightSqlServerBase::GetFlightInfoTables( + const ServerCallContext& context, const GetTables& command, + const FlightDescriptor& descriptor) { return Status::NotImplemented("GetFlightInfoTables not implemented"); } -Status FlightSqlServerBase::DoGetTables(const ServerCallContext& context, - const GetTables& command, - std::unique_ptr* result) { +arrow::Result> FlightSqlServerBase::DoGetTables( + const ServerCallContext& context, const GetTables& command) { return Status::NotImplemented("DoGetTables not implemented"); } -Status FlightSqlServerBase::GetFlightInfoTableTypes(const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { +arrow::Result> FlightSqlServerBase::GetFlightInfoTableTypes( + const ServerCallContext& context, const FlightDescriptor& descriptor) { return Status::NotImplemented("GetFlightInfoTableTypes not implemented"); } -Status FlightSqlServerBase::DoGetTableTypes(const ServerCallContext& context, - std::unique_ptr* result) { +arrow::Result> FlightSqlServerBase::DoGetTableTypes( + const ServerCallContext& context) { return Status::NotImplemented("DoGetTableTypes not implemented"); } -Status FlightSqlServerBase::GetFlightInfoPrimaryKeys(const ServerCallContext& context, - const GetPrimaryKeys& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { +arrow::Result> FlightSqlServerBase::GetFlightInfoPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command, + const FlightDescriptor& descriptor) { return Status::NotImplemented("GetFlightInfoPrimaryKeys not implemented"); } -Status FlightSqlServerBase::DoGetPrimaryKeys(const ServerCallContext& context, - const GetPrimaryKeys& command, - std::unique_ptr* result) { +arrow::Result> FlightSqlServerBase::DoGetPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command) { return Status::NotImplemented("DoGetPrimaryKeys not implemented"); } -Status FlightSqlServerBase::GetFlightInfoExportedKeys(const ServerCallContext& context, - const GetExportedKeys& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { +arrow::Result> FlightSqlServerBase::GetFlightInfoExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command, + const FlightDescriptor& descriptor) { return Status::NotImplemented("GetFlightInfoExportedKeys not implemented"); } -Status FlightSqlServerBase::DoGetExportedKeys(const ServerCallContext& context, - const GetExportedKeys& command, - std::unique_ptr* result) { +arrow::Result> FlightSqlServerBase::DoGetExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command) { return Status::NotImplemented("DoGetExportedKeys not implemented"); } -Status FlightSqlServerBase::GetFlightInfoImportedKeys(const ServerCallContext& context, - const GetImportedKeys& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info) { +arrow::Result> FlightSqlServerBase::GetFlightInfoImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command, + const FlightDescriptor& descriptor) { return Status::NotImplemented("GetFlightInfoImportedKeys not implemented"); } -Status FlightSqlServerBase::DoGetImportedKeys(const ServerCallContext& context, - const GetImportedKeys& command, - std::unique_ptr* result) { +arrow::Result> FlightSqlServerBase::DoGetImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command) { return Status::NotImplemented("DoGetImportedKeys not implemented"); } -Status FlightSqlServerBase::GetFlightInfoCrossReference( - const ServerCallContext& context, const GetCrossReference& command, - const FlightDescriptor& descriptor, std::unique_ptr* info) { +arrow::Result> +FlightSqlServerBase::GetFlightInfoCrossReference(const ServerCallContext& context, + const GetCrossReference& command, + const FlightDescriptor& descriptor) { return Status::NotImplemented("GetFlightInfoCrossReference not implemented"); } -Status FlightSqlServerBase::DoGetCrossReference( - const ServerCallContext& context, const GetCrossReference& command, - std::unique_ptr* result) { +arrow::Result> FlightSqlServerBase::DoGetCrossReference( + const ServerCallContext& context, const GetCrossReference& command) { return Status::NotImplemented("DoGetCrossReference not implemented"); } @@ -653,8 +669,8 @@ FlightSqlServerBase::CreatePreparedStatement( } Status FlightSqlServerBase::ClosePreparedStatement( - const ServerCallContext& context, const ActionClosePreparedStatementRequest& request, - std::unique_ptr* result) { + const ServerCallContext& context, + const ActionClosePreparedStatementRequest& request) { return Status::NotImplemented("ClosePreparedStatement not implemented"); } @@ -745,3 +761,5 @@ std::shared_ptr SqlSchema::GetSqlInfoSchema() { } // 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 index f1757fc27be..537cee7085b 100644 --- a/cpp/src/arrow/flight/sql/server.h +++ b/cpp/src/arrow/flight/sql/server.h @@ -150,245 +150,206 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { Status DoAction(const ServerCallContext& context, const Action& action, std::unique_ptr* result) override; - /// \brief Gets a FlightInfo for executing a SQL query. + /// \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. - /// \param[out] info The FlightInfo describing where to access the dataset. - /// \return Status. - virtual Status GetFlightInfoStatement(const ServerCallContext& context, - const StatementQuery& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info); - - /// \brief Gets a FlightDataStream containing the query results. + /// \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. - /// \param[out] result The FlightDataStream containing the results. - /// \return Status. - virtual Status DoGetStatement(const ServerCallContext& context, - const StatementQueryTicket& command, - std::unique_ptr* result); + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetStatement( + const ServerCallContext& context, const StatementQueryTicket& command); - /// \brief Gets a FlightInfo for executing an already created prepared statement. + /// \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. - /// \param[out] info The FlightInfo describing where to access the + /// \return The FlightInfo describing where to access the /// dataset. - /// \return Status. - virtual Status GetFlightInfoPreparedStatement(const ServerCallContext& context, - const PreparedStatementQuery& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual arrow::Result> GetFlightInfoPreparedStatement( + const ServerCallContext& context, const PreparedStatementQuery& command, + const FlightDescriptor& descriptor); - /// \brief Gets a FlightDataStream containing the prepared statement query results. + /// \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. - /// \param[out] result The FlightDataStream containing the results. - /// \return Status. - virtual Status DoGetPreparedStatement(const ServerCallContext& context, - const PreparedStatementQuery& command, - std::unique_ptr* result); + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetPreparedStatement( + const ServerCallContext& context, const PreparedStatementQuery& command); - /// \brief Gets a FlightInfo for listing catalogs. + /// \brief Get a FlightInfo for listing catalogs. /// \param[in] context Per-call context. /// \param[in] descriptor The descriptor identifying the data stream. - /// \param[out] info The FlightInfo describing where to access the dataset. - /// \return Status. - virtual Status GetFlightInfoCatalogs(const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + /// \return The FlightInfo describing where to access the dataset. + virtual arrow::Result> GetFlightInfoCatalogs( + const ServerCallContext& context, const FlightDescriptor& descriptor); - /// \brief Gets a FlightDataStream containing the list of catalogs. + /// \brief Get a FlightDataStream containing the list of catalogs. /// \param[in] context Per-call context. - /// \param[out] result An interface for sending data back to the client. - /// \return Status. - virtual Status DoGetCatalogs(const ServerCallContext& context, - std::unique_ptr* result); + /// \return An interface for sending data back to the client. + virtual arrow::Result> DoGetCatalogs( + const ServerCallContext& context); - /// \brief Gets a FlightInfo for retrieving other information (See SqlInfo). + /// \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. - /// \param[out] info The FlightInfo describing where to access the dataset. - /// \return Status. - virtual Status GetFlightInfoSqlInfo(const ServerCallContext& context, - const GetSqlInfo& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info); - - /// \brief Gets a FlightDataStream containing the list of SqlInfo results. + /// \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. - /// \param[out] result The FlightDataStream containing the results. - /// \return Status. - virtual Status DoGetSqlInfo(const ServerCallContext& context, const GetSqlInfo& command, - std::unique_ptr* result); + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetSqlInfo( + const ServerCallContext& context, const GetSqlInfo& command); - /// \brief Gets a FlightInfo for listing schemas. + /// \brief Get a FlightInfo for listing schemas. /// \param[in] context Per-call context. /// \param[in] command The GetSchemas object which may contain filters for /// catalog and schema name. /// \param[in] descriptor The descriptor identifying the data stream. - /// \param[out] info The FlightInfo describing where to access the dataset. - /// \return Status. - virtual Status GetFlightInfoSchemas(const ServerCallContext& context, - const GetSchemas& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info); - - /// \brief Gets a FlightDataStream containing the list of schemas. + /// \return The FlightInfo describing where to access the dataset. + virtual arrow::Result> GetFlightInfoSchemas( + const ServerCallContext& context, const GetSchemas& command, + const FlightDescriptor& descriptor); + + /// \brief Get a FlightDataStream containing the list of schemas. /// \param[in] context Per-call context. /// \param[in] command The GetSchemas object which may contain filters for /// catalog and schema name. - /// \param[out] result The FlightDataStream containing the results. - /// \return Status. - virtual Status DoGetSchemas(const ServerCallContext& context, const GetSchemas& command, - std::unique_ptr* result); + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetSchemas( + const ServerCallContext& context, const GetSchemas& command); - ///\brief Gets a FlightInfo for listing tables. + ///\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. - /// \param[out] info The FlightInfo describing where to access the dataset. - /// \return Status. - virtual Status GetFlightInfoTables(const ServerCallContext& context, - const GetTables& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info); - - /// \brief Gets a FlightDataStream containing the list of tables. + /// \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. - /// \param[out] result The FlightDataStream containing the results. - /// \return Status. - virtual Status DoGetTables(const ServerCallContext& context, const GetTables& command, - std::unique_ptr* result); + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetTables( + const ServerCallContext& context, const GetTables& command); - /// \brief Gets a FlightInfo to extract information about the table types. + /// \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. - /// \param[out] info The FlightInfo describing where to access the + /// \return The FlightInfo describing where to access the /// dataset. - /// \return Status. - virtual Status GetFlightInfoTableTypes(const ServerCallContext& context, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual arrow::Result> GetFlightInfoTableTypes( + const ServerCallContext& context, const FlightDescriptor& descriptor); - /// \brief Gets a FlightDataStream containing the data related to the table types. + /// \brief Get a FlightDataStream containing the data related to the table types. /// \param[in] context Per-call context. - /// \param[out] result The FlightDataStream containing the results. - /// \return Status. - virtual Status DoGetTableTypes(const ServerCallContext& context, - std::unique_ptr* result); + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetTableTypes( + const ServerCallContext& context); - /// \brief Gets a FlightInfo to extract information about primary and foreign keys. + /// \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. - /// \param[out] info The FlightInfo describing where to access the + /// \return The FlightInfo describing where to access the /// dataset. - /// \return Status. - virtual Status GetFlightInfoPrimaryKeys(const ServerCallContext& context, - const GetPrimaryKeys& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual arrow::Result> GetFlightInfoPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command, + const FlightDescriptor& descriptor); - /// \brief Gets a FlightDataStream containing the data related to the primary and + /// \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. - /// \param[out] result The FlightDataStream containing the results. - /// \return Status. - virtual Status DoGetPrimaryKeys(const ServerCallContext& context, - const GetPrimaryKeys& command, - std::unique_ptr* result); + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetPrimaryKeys( + const ServerCallContext& context, const GetPrimaryKeys& command); - /// \brief Gets a FlightInfo to extract information about foreign and primary keys. + /// \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. - /// \param[out] info The FlightInfo describing where to access the + /// \return The FlightInfo describing where to access the /// dataset. - /// \return Status. - virtual Status GetFlightInfoExportedKeys(const ServerCallContext& context, - const GetExportedKeys& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual arrow::Result> GetFlightInfoExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command, + const FlightDescriptor& descriptor); - /// \brief Gets a FlightDataStream containing the data related to the foreign and + /// \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. - /// \param[out] result The FlightDataStream containing the results. - /// \return Status. - virtual Status DoGetExportedKeys(const ServerCallContext& context, - const GetExportedKeys& command, - std::unique_ptr* result); + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetExportedKeys( + const ServerCallContext& context, const GetExportedKeys& command); - /// \brief Gets a FlightInfo to extract information about foreign and primary keys. + /// \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. - /// \param[out] info The FlightInfo describing where to access the + /// \return The FlightInfo describing where to access the /// dataset. - /// \return Status. - virtual Status GetFlightInfoImportedKeys(const ServerCallContext& context, - const GetImportedKeys& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual arrow::Result> GetFlightInfoImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command, + const FlightDescriptor& descriptor); - /// \brief Gets a FlightDataStream containing the data related to the foreign and + /// \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. - /// \param[out] result The FlightDataStream containing the results. - /// \return Status. - virtual Status DoGetImportedKeys(const ServerCallContext& context, - const GetImportedKeys& command, - std::unique_ptr* result); + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetImportedKeys( + const ServerCallContext& context, const GetImportedKeys& command); - /// \brief Gets a FlightInfo to extract information about foreign and primary keys. + /// \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. - /// \param[out] info The FlightInfo describing where to access the + /// \return The FlightInfo describing where to access the /// dataset. - /// \return Status. - virtual Status GetFlightInfoCrossReference(const ServerCallContext& context, - const GetCrossReference& command, - const FlightDescriptor& descriptor, - std::unique_ptr* info); + virtual arrow::Result> GetFlightInfoCrossReference( + const ServerCallContext& context, const GetCrossReference& command, + const FlightDescriptor& descriptor); - /// \brief Gets a FlightDataStream containing the data related to the foreign and + /// \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. - /// \param[out] result The FlightDataStream containing the results. - /// \return Status. - virtual Status DoGetCrossReference(const ServerCallContext& context, - const GetCrossReference& command, - std::unique_ptr* result); + /// \return The FlightDataStream containing the results. + virtual arrow::Result> DoGetCrossReference( + const ServerCallContext& context, const GetCrossReference& command); - /// \brief Executes an update SQL statement. + /// \brief Execute an update SQL statement. /// \param[in] context The call context. /// \param[in] command The StatementUpdate object containing the SQL statement. /// \param[in] reader a sequence of uploaded record batches. @@ -407,17 +368,15 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { const ServerCallContext& context, const ActionCreatePreparedStatementRequest& request); - /// \brief Closes a prepared statement. + /// \brief Close a prepared statement. /// \param[in] context The call context. /// \param[in] request The ActionClosePreparedStatementRequest object containing the /// prepared statement handle. - /// \param[out] result Empty ResultStream. virtual Status ClosePreparedStatement( const ServerCallContext& context, - const ActionClosePreparedStatementRequest& request, - std::unique_ptr* result); + const ActionClosePreparedStatementRequest& request); - /// \brief Binds parameters to given prepared statement. + /// \brief Bind parameters to given prepared statement. /// \param[in] context The call context. /// \param[in] command The PreparedStatementQuery object containing the /// prepared statement handle. @@ -428,7 +387,7 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { FlightMessageReader* reader, FlightMetadataWriter* writer); - /// \brief Executes an update SQL prepared statement. + /// \brief Execute an update SQL prepared statement. /// \param[in] context The call context. /// \param[in] command The PreparedStatementUpdate object containing the /// prepared statement handle. @@ -438,7 +397,7 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { const ServerCallContext& context, const PreparedStatementUpdate& command, FlightMessageReader* reader); - /// \brief Registers a new SqlInfo result. + /// \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); @@ -447,46 +406,46 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \brief Auxiliary class containing all Schemas used on Flight SQL. class ARROW_EXPORT SqlSchema { public: - /// \brief Gets the Schema used on GetCatalogs response. + /// \brief Get the Schema used on GetCatalogs response. /// \return The default schema template. static std::shared_ptr GetCatalogsSchema(); - /// \brief Gets the Schema used on GetSchemas response. + /// \brief Get the Schema used on GetSchemas response. /// \return The default schema template. static std::shared_ptr GetSchemasSchema(); - /// \brief Gets the Schema used on GetTables response when included schema + /// \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 Gets the Schema used on GetTables response when included schema + /// \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 Gets the Schema used on GetTableTypes response. + /// \brief Get the Schema used on GetTableTypes response. /// \return The default schema template. static std::shared_ptr GetTableTypesSchema(); - /// \brief Gets the Schema used on GetPrimaryKeys response when included schema + /// \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 Gets the Schema used on GetImportedKeys response. + /// \brief Get the Schema used on GetImportedKeys response. /// \return The default schema template. static std::shared_ptr GetExportedKeysSchema(); - /// \brief Gets the Schema used on GetImportedKeys response. + /// \brief Get the Schema used on GetImportedKeys response. /// \return The default schema template. static std::shared_ptr GetImportedKeysSchema(); - /// \brief Gets the Schema used on GetCrossReference response. + /// \brief Get the Schema used on GetCrossReference response. /// \return The default schema template. static std::shared_ptr GetCrossReferenceSchema(); - /// \brief Gets the Schema used on GetSqlInfo response. + /// \brief Get the Schema used on GetSqlInfo response. /// \return The default schema template. static std::shared_ptr GetSqlInfoSchema(); }; diff --git a/cpp/src/arrow/flight/sql/server_test.cc b/cpp/src/arrow/flight/sql/server_test.cc index f9f8328132c..5cd23444fae 100644 --- a/cpp/src/arrow/flight/sql/server_test.cc +++ b/cpp/src/arrow/flight/sql/server_test.cc @@ -185,8 +185,17 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { } TEST(TestFlightSqlServer, TestCommandGetTables) { - ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetTables({}, nullptr, nullptr, - nullptr, false, nullptr)); + 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)); @@ -208,10 +217,17 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { } TEST(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({}, nullptr, nullptr, &table_filter_pattern, false, nullptr)); + 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)); @@ -231,11 +247,17 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { } TEST(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({}, nullptr, nullptr, nullptr, false, &table_types)); + 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)); @@ -249,11 +271,17 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { } TEST(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({}, nullptr, nullptr, nullptr, false, &table_types)); + 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)); @@ -274,10 +302,17 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { } TEST(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({}, nullptr, nullptr, &table_filter_pattern, true, nullptr)); + 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)); @@ -322,7 +357,11 @@ TEST(TestFlightSqlServer, TestCommandGetCatalogs) { } TEST(TestFlightSqlServer, TestCommandGetSchemas) { - ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetSchemas({}, NULLPTR, NULLPTR)); + FlightCallOptions options = {}; + std::string* catalog = nullptr; + std::string* schema_filter_pattern = nullptr; + ASSERT_OK_AND_ASSIGN(auto flight_info, + sql_client->GetSchemas(options, catalog, schema_filter_pattern)); ASSERT_OK_AND_ASSIGN(auto stream, sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); @@ -530,8 +569,12 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { } TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { - ASSERT_OK_AND_ASSIGN(auto flight_info, - sql_client->GetPrimaryKeys({}, nullptr, nullptr, "int%")); + FlightCallOptions options = {}; + std::string* catalog = nullptr; + std::string* schema = nullptr; + std::string table_filter = "int%"; + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetPrimaryKeys( + options, catalog, schema, table_filter)); ASSERT_OK_AND_ASSIGN(auto stream, sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); @@ -554,8 +597,12 @@ TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { } TEST(TestFlightSqlServer, TestCommandGetImportedKeys) { - ASSERT_OK_AND_ASSIGN(auto flight_info, - sql_client->GetImportedKeys({}, NULLPTR, NULLPTR, "intTable")); + FlightCallOptions options = {}; + std::string* catalog = nullptr; + std::string* schema = nullptr; + std::string table_filter = "intTable"; + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetImportedKeys( + options, catalog, schema, table_filter)); ASSERT_OK_AND_ASSIGN(auto stream, sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); @@ -586,8 +633,12 @@ TEST(TestFlightSqlServer, TestCommandGetImportedKeys) { } TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { - ASSERT_OK_AND_ASSIGN(auto flight_info, - sql_client->GetExportedKeys({}, NULLPTR, NULLPTR, "foreignTable")); + FlightCallOptions options = {}; + std::string* catalog = nullptr; + std::string* schema = nullptr; + std::string table_filter = "foreignTable"; + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetExportedKeys( + options, catalog, schema, table_filter)); ASSERT_OK_AND_ASSIGN(auto stream, sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); @@ -618,9 +669,16 @@ TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { } TEST(TestFlightSqlServer, TestCommandGetCrossReference) { - ASSERT_OK_AND_ASSIGN(auto flight_info, - sql_client->GetCrossReference({}, NULLPTR, NULLPTR, "foreignTable", - NULLPTR, NULLPTR, "intTable")); + FlightCallOptions options = {}; + std::string* pk_catalog = nullptr; + std::string* pk_schema = nullptr; + std::string pk_table = "foreignTable"; + std::string* fk_catalog = nullptr; + std::string* fk_schema = nullptr; + std::string fk_table = "intTable"; + ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetCrossReference( + options, pk_catalog, pk_schema, pk_table, + fk_catalog, fk_schema, fk_table)); ASSERT_OK_AND_ASSIGN(auto stream, sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); From c36b81706a9d6260a733b5ad48baa26694759ddc Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 26 Nov 2021 13:25:38 -0300 Subject: [PATCH 223/237] Remove PRECOMPILED_HEADERS option on arrow_flight_sql's CMakeList.txt --- cpp/src/arrow/flight/sql/CMakeLists.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/cpp/src/arrow/flight/sql/CMakeLists.txt b/cpp/src/arrow/flight/sql/CMakeLists.txt index 656b1123dcf..edff4d85954 100644 --- a/cpp/src/arrow/flight/sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/sql/CMakeLists.txt @@ -48,8 +48,6 @@ add_arrow_lib(arrow_flight_sql ARROW_FLIGHT_SQL_LIBRARIES SOURCES ${ARROW_FLIGHT_SQL_SRCS} - PRECOMPILED_HEADERS - "$<$:arrow/flight/sql/pch.h>" DEPENDENCIES flight_sql_protobuf_gen SHARED_LINK_FLAGS From e8d8a13aa82c0ec929a103dcc81f250ab0dda02b Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 26 Nov 2021 13:59:04 -0300 Subject: [PATCH 224/237] [C++] Address ratification comments (round 4 - part 3) (#214) * Make other methods from SQLite server example to return arrow::Result instead of Status * Fix bug for null values in numeric columns on SQLite server example * Add comment regarding to performance on sqlite_statement_batch_reader * Separate GetSqlInfoResultMap from sqlite_server.h * Remove unused parameter on DoPutCommandStatementUpdate --- cpp/src/arrow/flight/sql/CMakeLists.txt | 2 + .../arrow/flight/sql/example/sqlite_server.cc | 33 +-- .../arrow/flight/sql/example/sqlite_server.h | 198 +-------------- .../flight/sql/example/sqlite_sql_info.cc | 225 ++++++++++++++++++ .../flight/sql/example/sqlite_sql_info.h | 34 +++ .../flight/sql/example/sqlite_statement.cc | 31 +-- .../flight/sql/example/sqlite_statement.h | 20 +- .../example/sqlite_statement_batch_reader.cc | 58 +++-- cpp/src/arrow/flight/sql/server.cc | 5 +- cpp/src/arrow/flight/sql/server.h | 4 +- cpp/src/arrow/flight/sql/server_test.cc | 33 +-- 11 files changed, 349 insertions(+), 294 deletions(-) create mode 100644 cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc create mode 100644 cpp/src/arrow/flight/sql/example/sqlite_sql_info.h diff --git a/cpp/src/arrow/flight/sql/CMakeLists.txt b/cpp/src/arrow/flight/sql/CMakeLists.txt index edff4d85954..0c80d193329 100644 --- a/cpp/src/arrow/flight/sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/sql/CMakeLists.txt @@ -70,6 +70,7 @@ add_arrow_test(flight_sql_test SOURCES client_test.cc server_test.cc + example/sqlite_sql_info.cc STATIC_LINK_LIBS ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} LABELS @@ -81,6 +82,7 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) add_executable(flight_sql_test_server test_server_cli.cc + example/sqlite_sql_info.cc example/sqlite_statement.cc example/sqlite_statement_batch_reader.cc example/sqlite_server.cc diff --git a/cpp/src/arrow/flight/sql/example/sqlite_server.cc b/cpp/src/arrow/flight/sql/example/sqlite_server.cc index cac80163d71..de2bd5b7e02 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.cc @@ -24,6 +24,7 @@ #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" @@ -204,6 +205,7 @@ arrow::Result> SQLiteFlightSqlServer::Cre 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; @@ -251,14 +253,11 @@ arrow::Result> SQLiteFlightSqlServer::GetFlightInfoS const FlightDescriptor& descriptor) { const std::string& query = command.query; - std::shared_ptr statement; - ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, query)); + ARROW_ASSIGN_OR_RAISE(auto statement, SqliteStatement::Create(db_, query)); - std::shared_ptr schema; - ARROW_RETURN_NOT_OK(statement->GetSchema(&schema)); + ARROW_ASSIGN_OR_RAISE(auto schema, statement->GetSchema()); - std::string ticket_string; - ARROW_ASSIGN_OR_RAISE(ticket_string, CreateStatementQueryTicket(query)); + 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)) @@ -363,17 +362,12 @@ arrow::Result> SQLiteFlightSqlServer::DoGetTab } arrow::Result SQLiteFlightSqlServer::DoPutCommandStatementUpdate( - const ServerCallContext& context, const StatementUpdate& command, - std::unique_ptr& reader) { + const ServerCallContext& context, const StatementUpdate& command) { const std::string& sql = command.query; - std::shared_ptr statement; - ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, sql)); - - int64_t record_count; - ARROW_RETURN_NOT_OK(statement->ExecuteUpdate(&record_count)); + ARROW_ASSIGN_OR_RAISE(auto statement, SqliteStatement::Create(db_, sql)); - return record_count; + return statement->ExecuteUpdate(); } arrow::Result @@ -385,8 +379,7 @@ SQLiteFlightSqlServer::CreatePreparedStatement( boost::uuids::uuid uuid = uuid_generator_(); prepared_statements_[uuid] = statement; - std::shared_ptr dataset_schema; - ARROW_RETURN_NOT_OK(statement->GetSchema(&dataset_schema)); + ARROW_ASSIGN_OR_RAISE(auto dataset_schema, statement->GetSchema()); sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); const int parameter_count = sqlite3_bind_parameter_count(stmt); @@ -449,8 +442,7 @@ SQLiteFlightSqlServer::GetFlightInfoPreparedStatement( std::shared_ptr statement = search->second; - std::shared_ptr schema; - ARROW_RETURN_NOT_OK(statement->GetSchema(&schema)); + ARROW_ASSIGN_OR_RAISE(auto schema, statement->GetSchema()); return GetFlightInfoForCommand(descriptor, schema); } @@ -497,10 +489,7 @@ arrow::Result SQLiteFlightSqlServer::DoPutPreparedStatementUpdate( sqlite3_stmt* stmt = statement->GetSqlite3Stmt(); ARROW_RETURN_NOT_OK(SetParametersOnSQLiteStatement(stmt, reader)); - int64_t record_count; - ARROW_RETURN_NOT_OK(statement->ExecuteUpdate(&record_count)); - - return record_count; + return statement->ExecuteUpdate(); } arrow::Result> SQLiteFlightSqlServer::GetFlightInfoTableTypes( diff --git a/cpp/src/arrow/flight/sql/example/sqlite_server.h b/cpp/src/arrow/flight/sql/example/sqlite_server.h index 58f88d9d5d3..f505c291e39 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.h @@ -26,7 +26,6 @@ #include #include "arrow/api.h" -#include "arrow/flight/sql/FlightSql.pb.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" @@ -36,200 +35,6 @@ namespace flight { namespace sql { namespace example { -namespace flight_sql_pb = arrow::flight::protocol::sql; - -/// \brief Gets the mapping from SQL info ids to SqlInfoResult instances. -/// \return the cache. -inline SqlInfoResultMap GetSqlInfoResultMap() { - return { - {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_NAME, - SqlInfoResult(std::string("db_name"))}, - {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_VERSION, - SqlInfoResult(std::string("sqlite 3"))}, - {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_ARROW_VERSION, - SqlInfoResult(std::string("7.0.0-SNAPSHOT" /* Only an example */))}, - {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_READ_ONLY, SqlInfoResult(false)}, - {flight_sql_pb::SqlInfo::SQL_DDL_CATALOG, - SqlInfoResult(false /* SQLite 3 does not support catalogs */)}, - {flight_sql_pb::SqlInfo::SQL_DDL_SCHEMA, - SqlInfoResult(false /* SQLite 3 does not support schemas */)}, - {flight_sql_pb::SqlInfo::SQL_DDL_TABLE, SqlInfoResult(true)}, - {flight_sql_pb::SqlInfo::SQL_IDENTIFIER_CASE, - SqlInfoResult(int64_t(flight_sql_pb::SqlSupportedCaseSensitivity:: - SQL_CASE_SENSITIVITY_CASE_INSENSITIVE))}, - {flight_sql_pb::SqlInfo::SQL_IDENTIFIER_QUOTE_CHAR, - SqlInfoResult(std::string("\""))}, - {flight_sql_pb::SqlInfo::SQL_QUOTED_IDENTIFIER_CASE, - SqlInfoResult(int64_t(flight_sql_pb::SqlSupportedCaseSensitivity:: - SQL_CASE_SENSITIVITY_CASE_INSENSITIVE))}, - {flight_sql_pb::SqlInfo::SQL_ALL_TABLES_ARE_SELECTABLE, SqlInfoResult(true)}, - {flight_sql_pb::SqlInfo::SQL_NULL_ORDERING, - SqlInfoResult(int64_t(flight_sql_pb::SqlNullOrdering::SQL_NULLS_SORTED_AT_START))}, - {flight_sql_pb::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"}))}, - {flight_sql_pb::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"}))}, - {flight_sql_pb::SqlInfo::SQL_STRING_FUNCTIONS, - SqlInfoResult( - std::vector({"SUBSTR", "TRIM", "LTRIM", "RTRIM", "LENGTH", - "REPLACE", "UPPER", "LOWER", "INSTR"}))}, - {flight_sql_pb::SqlInfo::SQL_SUPPORTS_CONVERT, - SqlInfoResult(std::unordered_map>( - {{flight_sql_pb::SqlSupportsConvert::SQL_CONVERT_BIGINT, - std::vector( - {flight_sql_pb::SqlSupportsConvert::SQL_CONVERT_INTEGER})}}))}}; -} - /// \brief Convert a column type to a ArrowType. /// \param sqlite_type the sqlite type. /// \return The equivalent ArrowType. @@ -274,8 +79,7 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { arrow::Result> DoGetSchemas( const ServerCallContext& context, const GetSchemas& command) override; arrow::Result DoPutCommandStatementUpdate( - const ServerCallContext& context, const StatementUpdate& update, - std::unique_ptr& reader) override; + const ServerCallContext& context, const StatementUpdate& update) override; arrow::Result CreatePreparedStatement( const ServerCallContext& context, const ActionCreatePreparedStatementRequest& request) override; 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..92007fa9eb3 --- /dev/null +++ b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc @@ -0,0 +1,225 @@ +// 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/FlightSql.pb.h" +#include "arrow/flight/sql/sql_info_types.h" + +namespace arrow { +namespace flight { +namespace sql { +namespace example { + +namespace flight_sql_pb = arrow::flight::protocol::sql; + +/// \brief Gets the mapping from SQL info ids to SqlInfoResult instances. +/// \return the cache. +SqlInfoResultMap GetSqlInfoResultMap() { + return { + {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_NAME, + SqlInfoResult(std::string("db_name"))}, + {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_VERSION, + SqlInfoResult(std::string("sqlite 3"))}, + {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_ARROW_VERSION, + SqlInfoResult(std::string("7.0.0-SNAPSHOT" /* Only an example */))}, + {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_READ_ONLY, SqlInfoResult(false)}, + {flight_sql_pb::SqlInfo::SQL_DDL_CATALOG, + SqlInfoResult(false /* SQLite 3 does not support catalogs */)}, + {flight_sql_pb::SqlInfo::SQL_DDL_SCHEMA, + SqlInfoResult(false /* SQLite 3 does not support schemas */)}, + {flight_sql_pb::SqlInfo::SQL_DDL_TABLE, SqlInfoResult(true)}, + {flight_sql_pb::SqlInfo::SQL_IDENTIFIER_CASE, + SqlInfoResult(int64_t(flight_sql_pb::SqlSupportedCaseSensitivity:: + SQL_CASE_SENSITIVITY_CASE_INSENSITIVE))}, + {flight_sql_pb::SqlInfo::SQL_IDENTIFIER_QUOTE_CHAR, + SqlInfoResult(std::string("\""))}, + {flight_sql_pb::SqlInfo::SQL_QUOTED_IDENTIFIER_CASE, + SqlInfoResult(int64_t(flight_sql_pb::SqlSupportedCaseSensitivity:: + SQL_CASE_SENSITIVITY_CASE_INSENSITIVE))}, + {flight_sql_pb::SqlInfo::SQL_ALL_TABLES_ARE_SELECTABLE, SqlInfoResult(true)}, + {flight_sql_pb::SqlInfo::SQL_NULL_ORDERING, + SqlInfoResult(int64_t(flight_sql_pb::SqlNullOrdering::SQL_NULLS_SORTED_AT_START))}, + {flight_sql_pb::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"}))}, + {flight_sql_pb::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"}))}, + {flight_sql_pb::SqlInfo::SQL_STRING_FUNCTIONS, + SqlInfoResult( + std::vector({"SUBSTR", "TRIM", "LTRIM", "RTRIM", "LENGTH", + "REPLACE", "UPPER", "LOWER", "INSTR"}))}, + {flight_sql_pb::SqlInfo::SQL_SUPPORTS_CONVERT, + SqlInfoResult(std::unordered_map>( + {{flight_sql_pb::SqlSupportsConvert::SQL_CONVERT_BIGINT, + std::vector( + {flight_sql_pb::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..f6057540774 --- /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/sql_info_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 index 3d7c5efeb23..018f8de37db 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_statement.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_statement.cc @@ -66,7 +66,7 @@ arrow::Result> SqliteStatement::Create( return result; } -Status SqliteStatement::GetSchema(std::shared_ptr* schema) const { +arrow::Result> SqliteStatement::GetSchema() const { std::vector> fields; int column_count = sqlite3_column_count(stmt_); for (int i = 0; i < column_count; i++) { @@ -99,41 +99,36 @@ Status SqliteStatement::GetSchema(std::shared_ptr* schema) const { fields.push_back(arrow::field(column_name, data_type)); } - *schema = arrow::schema(fields); - return Status::OK(); + return arrow::schema(fields); } SqliteStatement::~SqliteStatement() { sqlite3_finalize(stmt_); } -Status SqliteStatement::Step(int* rc) { - *rc = sqlite3_step(stmt_); - if (*rc == SQLITE_ERROR) { +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 Status::OK(); + return rc; } -Status SqliteStatement::Reset(int* rc) { - *rc = sqlite3_reset(stmt_); - if (*rc == SQLITE_ERROR) { +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 Status::OK(); + return rc; } sqlite3_stmt* SqliteStatement::GetSqlite3Stmt() const { return stmt_; } -Status SqliteStatement::ExecuteUpdate(int64_t* result) { - int rc; - ARROW_RETURN_NOT_OK(Step(&rc)); - - *result = sqlite3_changes(db_); - - return Status::OK(); +arrow::Result SqliteStatement::ExecuteUpdate() { + ARROW_RETURN_NOT_OK(Step()); + return sqlite3_changes(db_); } } // namespace example diff --git a/cpp/src/arrow/flight/sql/example/sqlite_statement.h b/cpp/src/arrow/flight/sql/example/sqlite_statement.h index 0d32a2f282d..a3f086abc47 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_statement.h +++ b/cpp/src/arrow/flight/sql/example/sqlite_statement.h @@ -41,28 +41,24 @@ class SqliteStatement { ~SqliteStatement(); /// \brief Creates an Arrow Schema based on the results of this statement. - /// \param[out] schema The resulting Schema. - /// \return Status. - Status GetSchema(std::shared_ptr* schema) const; + /// \return The resulting Schema. + arrow::Result> GetSchema() const; /// \brief Steps on underlying sqlite3_stmt. - /// \param[out] rc The resulting return code from SQLite. - /// \return Status. - Status Step(int* rc); + /// \return The resulting return code from SQLite. + arrow::Result Step(); /// \brief Reset the state of the sqlite3_stmt. - /// \param[out] rc The resulting return code from SQLite. - /// \return Status. - Status Reset(int* rc); + /// \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. - /// \param[out] result The number of rows changed by execution. - /// \return Status. - Status ExecuteUpdate(int64_t* result); + /// \return The number of rows changed by execution. + arrow::Result ExecuteUpdate(); private: sqlite3* db_; 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 index c6ac34bfddb..a5824ae255f 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_statement_batch_reader.cc @@ -50,29 +50,40 @@ break; \ } -#define INT_BUILDER_CASE(TYPE_CLASS, STMT, COLUMN) \ - case TYPE_CLASS##Type::type_id: { \ - sqlite3_int64 value = sqlite3_column_int64(STMT, COLUMN); \ - ARROW_RETURN_NOT_OK( \ - (reinterpret_cast(builder)).Append(value)); \ - 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: { \ - double value = sqlite3_column_double(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; \ } -#define MAX_BATCH_SIZE 1024 - 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( @@ -84,11 +95,9 @@ SqliteStatementBatchReader::SqliteStatementBatchReader( Result> SqliteStatementBatchReader::Create( const std::shared_ptr& statement_) { - int rc; - ARROW_RETURN_NOT_OK(statement_->Step(&rc)); + ARROW_RETURN_NOT_OK(statement_->Step()); - std::shared_ptr schema; - ARROW_RETURN_NOT_OK(statement_->GetSchema(&schema)); + ARROW_ASSIGN_OR_RAISE(auto schema, statement_->GetSchema()); std::shared_ptr result( new SqliteStatementBatchReader(statement_, schema)); @@ -119,19 +128,22 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr* out) { } if (!already_executed_) { - ARROW_RETURN_NOT_OK(statement_->Reset(&rc_)); - ARROW_RETURN_NOT_OK(statement_->Step(&rc_)); + ARROW_ASSIGN_OR_RAISE(rc_, statement_->Reset()); + ARROW_ASSIGN_OR_RAISE(rc_, statement_->Step()); already_executed_ = true; } - int rows = 0; - while (rows < MAX_BATCH_SIZE && rc_ == SQLITE_ROW) { + 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) @@ -154,7 +166,7 @@ Status SqliteStatementBatchReader::ReadNext(std::shared_ptr* out) { } } - ARROW_RETURN_NOT_OK(statement_->Step(&rc_)); + ARROW_ASSIGN_OR_RAISE(rc_, statement_->Step()); } if (rows > 0) { diff --git a/cpp/src/arrow/flight/sql/server.cc b/cpp/src/arrow/flight/sql/server.cc index 219ffd8e01b..6a34bdeb96a 100644 --- a/cpp/src/arrow/flight/sql/server.cc +++ b/cpp/src/arrow/flight/sql/server.cc @@ -402,7 +402,7 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, ARROW_ASSIGN_OR_RAISE(StatementUpdate internal_command, ParseCommandStatementUpdate(any)); ARROW_ASSIGN_OR_RAISE(auto record_count, - DoPutCommandStatementUpdate(context, internal_command, reader)) + DoPutCommandStatementUpdate(context, internal_command)) pb::sql::DoPutUpdateResult result; result.set_record_count(record_count); @@ -687,8 +687,7 @@ arrow::Result FlightSqlServerBase::DoPutPreparedStatementUpdate( } arrow::Result FlightSqlServerBase::DoPutCommandStatementUpdate( - const ServerCallContext& context, const StatementUpdate& command, - std::unique_ptr& reader) { + const ServerCallContext& context, const StatementUpdate& command) { return Status::NotImplemented("DoPutCommandStatementUpdate not implemented"); } diff --git a/cpp/src/arrow/flight/sql/server.h b/cpp/src/arrow/flight/sql/server.h index 537cee7085b..f63606e557a 100644 --- a/cpp/src/arrow/flight/sql/server.h +++ b/cpp/src/arrow/flight/sql/server.h @@ -352,11 +352,9 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \brief Execute an update SQL statement. /// \param[in] context The call context. /// \param[in] command The StatementUpdate object containing the SQL statement. - /// \param[in] reader a sequence of uploaded record batches. /// \return The changed record count. virtual arrow::Result DoPutCommandStatementUpdate( - const ServerCallContext& context, const StatementUpdate& command, - std::unique_ptr& reader); + const ServerCallContext& context, const StatementUpdate& command); /// \brief Create a prepared statement from given SQL statement. /// \param[in] context The call context. diff --git a/cpp/src/arrow/flight/sql/server_test.cc b/cpp/src/arrow/flight/sql/server_test.cc index 5cd23444fae..30466eb32f1 100644 --- a/cpp/src/arrow/flight/sql/server_test.cc +++ b/cpp/src/arrow/flight/sql/server_test.cc @@ -23,6 +23,7 @@ #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" @@ -30,8 +31,6 @@ using ::testing::_; using ::testing::Ref; -namespace pb = arrow::flight::protocol; - using arrow::internal::checked_cast; namespace arrow { @@ -173,10 +172,11 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { 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])"); - const auto keyname_array = ArrayFromJSON(utf8(), R"(["one", "zero", "negative one"])"); - const auto value_array = ArrayFromJSON(int64(), R"([1, 0, -1])"); - const auto foreignId_array = ArrayFromJSON(int64(), R"([1, 1, 1])"); + 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}); @@ -428,10 +428,11 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQuery) { 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])"); - const auto keyname_array = ArrayFromJSON(utf8(), R"(["one", "zero", "negative one"])"); - const auto value_array = ArrayFromJSON(int64(), R"([1, 0, -1])"); - const auto foreignId_array = ArrayFromJSON(int64(), R"([1, 1, 1])"); + 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}); @@ -538,16 +539,16 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding ASSERT_OK(prepared_statement->SetParameters(record_batch)); - ASSERT_OK_AND_EQ(3, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); + ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); ASSERT_OK_AND_EQ(1, prepared_statement->ExecuteUpdate()); - ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); + 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(3, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); + ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); } TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { @@ -556,16 +557,16 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { sql_client->Prepare( {}, "INSERT INTO INTTABLE (keyName, value) VALUES ('new_value', 999)")); - ASSERT_OK_AND_EQ(3, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); + ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); ASSERT_OK_AND_EQ(1, prepared_statement->ExecuteUpdate()); - ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); + 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(3, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); + ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); } TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { From fe9d7dc5a1b26a00c789cee969e750479353f581 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 2 Dec 2021 13:28:23 -0300 Subject: [PATCH 225/237] Make sure to wait for server to be ready before running tests (#220) * Make sure to wait for server to be ready before running tests * Start server independently for each test * Use unique_ptr for server thread on server_test.cc --- cpp/src/arrow/flight/sql/CMakeLists.txt | 40 ++++--- cpp/src/arrow/flight/sql/server_test.cc | 136 +++++++++++++++--------- 2 files changed, 103 insertions(+), 73 deletions(-) diff --git a/cpp/src/arrow/flight/sql/CMakeLists.txt b/cpp/src/arrow/flight/sql/CMakeLists.txt index 0c80d193329..178fec9f3e5 100644 --- a/cpp/src/arrow/flight/sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/sql/CMakeLists.txt @@ -66,27 +66,29 @@ else() ${ARROW_TEST_LINK_LIBS}) endif() -add_arrow_test(flight_sql_test - SOURCES - client_test.cc - server_test.cc - example/sqlite_sql_info.cc - STATIC_LINK_LIBS - ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} - LABELS - "arrow_flight_sql") - # Build test server for unit tests if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) find_package(SQLite3Alt REQUIRED) - add_executable(flight_sql_test_server - test_server_cli.cc - 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) + set(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 + ${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 ${flight_sql_test_server_SRCS}) target_link_libraries(flight_sql_test_server PRIVATE ${ARROW_FLIGHT_SQL_TEST_LINK_LIBS} ${GFLAGS_LIBRARIES} ${SQLite3_LIBRARIES}) @@ -94,8 +96,4 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) 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}) - - if(ARROW_BUILD_TESTS) - add_dependencies(arrow-flight-sql-test flight_sql_test_server) - endif() endif() diff --git a/cpp/src/arrow/flight/sql/server_test.cc b/cpp/src/arrow/flight/sql/server_test.cc index 30466eb32f1..6db9548046c 100644 --- a/cpp/src/arrow/flight/sql/server_test.cc +++ b/cpp/src/arrow/flight/sql/server_test.cc @@ -17,9 +17,13 @@ #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" @@ -123,24 +127,35 @@ class SqlInfoDenseUnionValidator { SqlInfoDenseUnionValidator& operator=(const SqlInfoDenseUnionValidator&) = delete; }; -std::unique_ptr server; -std::unique_ptr sql_client; +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; + } -class TestFlightSqlServer : public ::testing::Environment { protected: void SetUp() override { - server.reset(new TestServer("flight_sql_test_server")); - server->Start(); - for (int i = 0; i < 100; i++) { - std::this_thread::sleep_for(std::chrono::milliseconds(10)); - if (server->IsRunning()) { - break; - } - } - ASSERT_TRUE(server->IsRunning()); + unix_sock = std::tmpnam(nullptr); + server_thread.reset(new std::thread(RunServer, this)); + + std::unique_lock lk(server_ready_m); + server_ready_cv.wait(lk); std::stringstream ss; - ss << "grpc://localhost:" << server->port(); + ss << "grpc+unix://" << unix_sock; std::string uri = ss.str(); std::unique_ptr client; @@ -153,12 +168,46 @@ class TestFlightSqlServer : public ::testing::Environment { void TearDown() override { sql_client.reset(); - server->Stop(); - server.reset(); + + ASSERT_OK(server->Shutdown()); + server_thread->join(); + server_thread.reset(); + } + + private: + std::string unix_sock; + std::shared_ptr server; + std::unique_ptr server_thread; + std::condition_variable server_ready_cv; + std::mutex server_ready_m; + + Status RunServerInternal() { + arrow::flight::Location location; + ARROW_CHECK_OK(arrow::flight::Location::ForGrpcUnix(unix_sock, &location)); + arrow::flight::FlightServerOptions options(location); + + 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})); + + server_ready_cv.notify_all(); + ARROW_CHECK_OK(server->Serve()); + + return arrow::Status::OK(); + } + + static void RunServer(TestFlightSqlServer* server) { + arrow::Status st = server->RunServerInternal(); + if (!st.ok()) { + std::cerr << st << std::endl; + } } }; -TEST(TestFlightSqlServer, TestCommandStatementQuery) { +TEST_F(TestFlightSqlServer, TestCommandStatementQuery) { ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->Execute({}, "SELECT * FROM intTable")); @@ -184,7 +233,7 @@ TEST(TestFlightSqlServer, TestCommandStatementQuery) { AssertTablesEqual(*expected_table, *table); } -TEST(TestFlightSqlServer, TestCommandGetTables) { +TEST_F(TestFlightSqlServer, TestCommandGetTables) { FlightCallOptions options = {}; std::string* catalog = nullptr; std::string* schema_filter_pattern = nullptr; @@ -216,7 +265,7 @@ TEST(TestFlightSqlServer, TestCommandGetTables) { AssertTablesEqual(*expected_table, *table); } -TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { +TEST_F(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { FlightCallOptions options = {}; std::string* catalog = nullptr; std::string* schema_filter_pattern = nullptr; @@ -246,7 +295,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableFilter) { AssertTablesEqual(*expected_table, *table); } -TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { +TEST_F(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { FlightCallOptions options = {}; std::string* catalog = nullptr; std::string* schema_filter_pattern = nullptr; @@ -270,7 +319,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithTableTypesFilter) { ASSERT_EQ(table->num_rows(), 0); } -TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { +TEST_F(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { FlightCallOptions options = {}; std::string* catalog = nullptr; std::string* schema_filter_pattern = nullptr; @@ -301,7 +350,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithUnexistenceTableTypeFilter) { AssertTablesEqual(*expected_table, *table); } -TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { +TEST_F(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { FlightCallOptions options = {}; std::string* catalog = nullptr; std::string* schema_filter_pattern = nullptr; @@ -341,7 +390,7 @@ TEST(TestFlightSqlServer, TestCommandGetTablesWithIncludedSchemas) { AssertTablesEqual(*expected_table, *table); } -TEST(TestFlightSqlServer, TestCommandGetCatalogs) { +TEST_F(TestFlightSqlServer, TestCommandGetCatalogs) { ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetCatalogs({})); ASSERT_OK_AND_ASSIGN(auto stream, @@ -356,7 +405,7 @@ TEST(TestFlightSqlServer, TestCommandGetCatalogs) { ASSERT_EQ(0, table->num_rows()); } -TEST(TestFlightSqlServer, TestCommandGetSchemas) { +TEST_F(TestFlightSqlServer, TestCommandGetSchemas) { FlightCallOptions options = {}; std::string* catalog = nullptr; std::string* schema_filter_pattern = nullptr; @@ -375,7 +424,7 @@ TEST(TestFlightSqlServer, TestCommandGetSchemas) { ASSERT_EQ(0, table->num_rows()); } -TEST(TestFlightSqlServer, TestCommandGetTableTypes) { +TEST_F(TestFlightSqlServer, TestCommandGetTableTypes) { ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetTableTypes({})); ASSERT_OK_AND_ASSIGN(auto stream, @@ -391,7 +440,7 @@ TEST(TestFlightSqlServer, TestCommandGetTableTypes) { AssertTablesEqual(*expected_table, *table); } -TEST(TestFlightSqlServer, TestCommandStatementUpdate) { +TEST_F(TestFlightSqlServer, TestCommandStatementUpdate) { int64_t result; ASSERT_OK_AND_ASSIGN(result, sql_client->ExecuteUpdate( @@ -412,7 +461,7 @@ TEST(TestFlightSqlServer, TestCommandStatementUpdate) { ASSERT_EQ(3, result); } -TEST(TestFlightSqlServer, TestCommandPreparedStatementQuery) { +TEST_F(TestFlightSqlServer, TestCommandPreparedStatementQuery) { ASSERT_OK_AND_ASSIGN(auto prepared_statement, sql_client->Prepare({}, "SELECT * FROM intTable")); @@ -440,7 +489,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQuery) { AssertTablesEqual(*expected_table, *table); } -TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) { +TEST_F(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) { ASSERT_OK_AND_ASSIGN( auto prepared_statement, sql_client->Prepare({}, "SELECT * FROM intTable WHERE keyName LIKE ?")); @@ -493,22 +542,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementQueryWithParameterBinding) AssertTablesEqual(*expected_table, *table); } -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; -} - -TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding) { +TEST_F(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding) { ASSERT_OK_AND_ASSIGN( auto prepared_statement, sql_client->Prepare( @@ -551,7 +585,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdateWithParameterBinding ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); } -TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { +TEST_F(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { ASSERT_OK_AND_ASSIGN( auto prepared_statement, sql_client->Prepare( @@ -569,7 +603,7 @@ TEST(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { ASSERT_OK_AND_EQ(4, ExecuteCountQuery("SELECT COUNT(*) FROM intTable")); } -TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { +TEST_F(TestFlightSqlServer, TestCommandGetPrimaryKeys) { FlightCallOptions options = {}; std::string* catalog = nullptr; std::string* schema = nullptr; @@ -597,7 +631,7 @@ TEST(TestFlightSqlServer, TestCommandGetPrimaryKeys) { AssertTablesEqual(*expected_table, *table); } -TEST(TestFlightSqlServer, TestCommandGetImportedKeys) { +TEST_F(TestFlightSqlServer, TestCommandGetImportedKeys) { FlightCallOptions options = {}; std::string* catalog = nullptr; std::string* schema = nullptr; @@ -633,7 +667,7 @@ TEST(TestFlightSqlServer, TestCommandGetImportedKeys) { AssertTablesEqual(*expected_table, *table); } -TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { +TEST_F(TestFlightSqlServer, TestCommandGetExportedKeys) { FlightCallOptions options = {}; std::string* catalog = nullptr; std::string* schema = nullptr; @@ -669,7 +703,7 @@ TEST(TestFlightSqlServer, TestCommandGetExportedKeys) { AssertTablesEqual(*expected_table, *table); } -TEST(TestFlightSqlServer, TestCommandGetCrossReference) { +TEST_F(TestFlightSqlServer, TestCommandGetCrossReference) { FlightCallOptions options = {}; std::string* pk_catalog = nullptr; std::string* pk_schema = nullptr; @@ -709,7 +743,7 @@ TEST(TestFlightSqlServer, TestCommandGetCrossReference) { AssertTablesEqual(*expected_table, *table); } -TEST(TestFlightSqlServer, TestCommandGetSqlInfo) { +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()); @@ -743,7 +777,7 @@ TEST(TestFlightSqlServer, TestCommandGetSqlInfo) { } } -TEST(TestFlightSqlServer, TestCommandGetSqlInfoNoInfo) { +TEST_F(TestFlightSqlServer, TestCommandGetSqlInfoNoInfo) { FlightCallOptions call_options; ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetSqlInfo(call_options, {999999})); auto result = sql_client->DoGet(call_options, flight_info->endpoints()[0].ticket); @@ -754,8 +788,6 @@ TEST(TestFlightSqlServer, TestCommandGetSqlInfoNoInfo) { ::testing::HasSubstr("No information for SQL info number 999999.")); } -auto env = ::testing::AddGlobalTestEnvironment(new TestFlightSqlServer); - } // namespace sql } // namespace flight } // namespace arrow From 8431e41bddda5abc8aeec8a363b40deab352e5a4 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 2 Dec 2021 13:35:56 -0300 Subject: [PATCH 226/237] [C++] Address ratification comments (round 4 - part 4) (#215) * Make other methods from SQLite server example to return arrow::Result instead of Status * Fix bug for null values in numeric columns on SQLite server example * Structure catalog-schema-table tuple on a TableRef struct on client * Rename 'schema' to 'db_schema' * Use TableRef struct on server.cc * Undo renaming db_schema_filter_pattern --- cpp/src/arrow/flight/sql/client.cc | 81 +++++++-------- cpp/src/arrow/flight/sql/client.h | 67 +++++-------- cpp/src/arrow/flight/sql/client_test.cc | 36 ++++--- .../arrow/flight/sql/example/sqlite_server.cc | 69 +++++++------ .../arrow/flight/sql/example/sqlite_server.h | 6 +- .../flight/sql/example/sqlite_sql_info.cc | 2 +- .../flight/sql/example/sqlite_sql_info.h | 2 +- cpp/src/arrow/flight/sql/server.cc | 73 +++++++------- cpp/src/arrow/flight/sql/server.h | 42 +++----- cpp/src/arrow/flight/sql/server_test.cc | 38 +++---- cpp/src/arrow/flight/sql/sql_info_internal.h | 2 +- cpp/src/arrow/flight/sql/test_app_cli.cc | 21 ++-- .../flight/sql/{sql_info_types.h => types.h} | 11 +++ format/FlightSql.proto | 98 +++++++++---------- 14 files changed, 265 insertions(+), 283 deletions(-) rename cpp/src/arrow/flight/sql/{sql_info_types.h => types.h} (78%) diff --git a/cpp/src/arrow/flight/sql/client.cc b/cpp/src/arrow/flight/sql/client.cc index 833000e479d..50a5777cd9c 100644 --- a/cpp/src/arrow/flight/sql/client.cc +++ b/cpp/src/arrow/flight/sql/client.cc @@ -116,15 +116,15 @@ arrow::Result> FlightSqlClient::GetCatalogs( return GetFlightInfoForCommand(*this, options, command); } -arrow::Result> FlightSqlClient::GetSchemas( +arrow::Result> FlightSqlClient::GetDbSchemas( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema_filter_pattern) { - flight_sql_pb::CommandGetSchemas command; + const std::string* db_schema_filter_pattern) { + flight_sql_pb::CommandGetDbSchemas command; if (catalog != NULLPTR) { command.set_catalog(*catalog); } - if (schema_filter_pattern != NULLPTR) { - command.set_schema_filter_pattern(*schema_filter_pattern); + if (db_schema_filter_pattern != NULLPTR) { + command.set_db_schema_filter_pattern(*db_schema_filter_pattern); } return GetFlightInfoForCommand(*this, options, command); @@ -132,7 +132,7 @@ arrow::Result> FlightSqlClient::GetSchemas( arrow::Result> FlightSqlClient::GetTables( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema_filter_pattern, const std::string* table_filter_pattern, + 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; @@ -140,8 +140,8 @@ arrow::Result> FlightSqlClient::GetTables( command.set_catalog(*catalog); } - if (schema_filter_pattern != NULLPTR) { - command.set_schema_filter_pattern(*schema_filter_pattern); + if (db_schema_filter_pattern != NULLPTR) { + command.set_db_schema_filter_pattern(*db_schema_filter_pattern); } if (table_filter_pattern != NULLPTR) { @@ -160,81 +160,76 @@ arrow::Result> FlightSqlClient::GetTables( } arrow::Result> FlightSqlClient::GetPrimaryKeys( - const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table) { + const FlightCallOptions& options, const TableRef& table_ref) { flight_sql_pb::CommandGetPrimaryKeys command; - if (catalog != NULLPTR) { - command.set_catalog(*catalog); + if (table_ref.catalog.has_value()) { + command.set_catalog(table_ref.catalog.value()); } - if (schema != NULLPTR) { - command.set_schema(*schema); + if (table_ref.db_schema.has_value()) { + command.set_db_schema(table_ref.db_schema.value()); } - command.set_table(table); + command.set_table(table_ref.table); return GetFlightInfoForCommand(*this, options, command); } arrow::Result> FlightSqlClient::GetExportedKeys( - const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table) { + const FlightCallOptions& options, const TableRef& table_ref) { flight_sql_pb::CommandGetExportedKeys command; - if (catalog != NULLPTR) { - command.set_catalog(*catalog); + if (table_ref.catalog.has_value()) { + command.set_catalog(table_ref.catalog.value()); } - if (schema != NULLPTR) { - command.set_schema(*schema); + if (table_ref.db_schema.has_value()) { + command.set_db_schema(table_ref.db_schema.value()); } - command.set_table(table); + command.set_table(table_ref.table); return GetFlightInfoForCommand(*this, options, command); } arrow::Result> FlightSqlClient::GetImportedKeys( - const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table) { + const FlightCallOptions& options, const TableRef& table_ref) { flight_sql_pb::CommandGetImportedKeys command; - if (catalog != NULLPTR) { - command.set_catalog(*catalog); + if (table_ref.catalog.has_value()) { + command.set_catalog(table_ref.catalog.value()); } - if (schema != NULLPTR) { - command.set_schema(*schema); + if (table_ref.db_schema.has_value()) { + command.set_db_schema(table_ref.db_schema.value()); } - command.set_table(table); + command.set_table(table_ref.table); return GetFlightInfoForCommand(*this, options, command); } arrow::Result> FlightSqlClient::GetCrossReference( - const FlightCallOptions& options, const std::string* pk_catalog, - const std::string* pk_schema, const std::string& pk_table, - const std::string* fk_catalog, const std::string* fk_schema, - const std::string& fk_table) { + const FlightCallOptions& options, const TableRef& pk_table_ref, + const TableRef& fk_table_ref) { flight_sql_pb::CommandGetCrossReference command; - if (pk_catalog != NULLPTR) { - command.set_pk_catalog(*pk_catalog); + if (pk_table_ref.catalog.has_value()) { + command.set_pk_catalog(pk_table_ref.catalog.value()); } - if (pk_schema != NULLPTR) { - command.set_pk_schema(*pk_schema); + 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); + command.set_pk_table(pk_table_ref.table); - if (fk_catalog != NULLPTR) { - command.set_fk_catalog(*fk_catalog); + if (fk_table_ref.catalog.has_value()) { + command.set_fk_catalog(fk_table_ref.catalog.value()); } - if (fk_schema != NULLPTR) { - command.set_fk_schema(*fk_schema); + 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); + command.set_fk_table(fk_table_ref.table); return GetFlightInfoForCommand(*this, options, command); } diff --git a/cpp/src/arrow/flight/sql/client.h b/cpp/src/arrow/flight/sql/client.h index 82ed35787e1..5bf1b3e64a0 100644 --- a/cpp/src/arrow/flight/sql/client.h +++ b/cpp/src/arrow/flight/sql/client.h @@ -21,6 +21,7 @@ #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" @@ -63,14 +64,14 @@ class ARROW_EXPORT FlightSqlClient { arrow::Result> GetCatalogs( const FlightCallOptions& options); - /// \brief Request a list of schemas. - /// \param[in] options RPC-layer hints for this call. - /// \param[in] catalog The catalog. - /// \param[in] schema_filter_pattern The schema filter pattern. + /// \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> GetSchemas( + arrow::Result> GetDbSchemas( const FlightCallOptions& options, const std::string* catalog, - const std::string* schema_filter_pattern); + 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 @@ -81,66 +82,52 @@ class ARROW_EXPORT FlightSqlClient { 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] 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. + /// \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* schema_filter_pattern, const std::string* table_filter_pattern, - bool include_schema, const std::vector* table_types); + 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] catalog The catalog. - /// \param[in] schema The schema. - /// \param[in] table The table. + /// \param[in] table_ref The table reference. /// \return The FlightInfo describing where to access the dataset. arrow::Result> GetPrimaryKeys( - const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table); + 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] catalog The foreign key table catalog. - /// \param[in] schema The foreign key table schema. - /// \param[in] table The foreign key table. Cannot be null. + /// \param[in] table_ref The table reference. /// \return The FlightInfo describing where to access the dataset. arrow::Result> GetExportedKeys( - const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table); + 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] catalog The primary key table catalog. - /// \param[in] schema The primary key table schema. - /// \param[in] table The primary key table. Cannot be null. + /// \param[in] table_ref The table reference. /// \return The FlightInfo describing where to access the dataset. arrow::Result> GetImportedKeys( - const FlightCallOptions& options, const std::string* catalog, - const std::string* schema, const std::string& table); + 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_catalog The catalog of the table that exports the key. - /// \param[in] pk_schema The schema of the table that exports the key. - /// \param[in] pk_table The table that exports the key. - /// \param[in] fk_catalog The catalog of the table that imports the key. - /// \param[in] fk_schema The schema of the table that imports the key. - /// \param[in] fk_table The table that imports the key. + /// \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 std::string* pk_catalog, - const std::string* pk_schema, const std::string& pk_table, - const std::string* fk_catalog, const std::string* fk_schema, - const std::string& fk_table); + 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. diff --git a/cpp/src/arrow/flight/sql/client_test.cc b/cpp/src/arrow/flight/sql/client_test.cc index b2b08c78ee7..8c0c8333074 100644 --- a/cpp/src/arrow/flight/sql/client_test.cc +++ b/cpp/src/arrow/flight/sql/client_test.cc @@ -135,19 +135,19 @@ TEST_F(TestFlightSqlClient, TestGetCatalogs) { ASSERT_OK(sql_client_.GetCatalogs(call_options_)); } -TEST_F(TestFlightSqlClient, TestGetSchemas) { +TEST_F(TestFlightSqlClient, TestGetDbSchemas) { std::string schema_filter_pattern = "schema_filter_pattern"; std::string catalog = "catalog"; - pb::sql::CommandGetSchemas command; + pb::sql::CommandGetDbSchemas command; command.set_catalog(catalog); - command.set_schema_filter_pattern(schema_filter_pattern); + 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_.GetSchemas(call_options_, &catalog, &schema_filter_pattern)); + ASSERT_OK(sql_client_.GetDbSchemas(call_options_, &catalog, &schema_filter_pattern)); } TEST_F(TestFlightSqlClient, TestGetTables) { @@ -159,7 +159,7 @@ TEST_F(TestFlightSqlClient, TestGetTables) { pb::sql::CommandGetTables command; command.set_catalog(catalog); - command.set_schema_filter_pattern(schema_filter_pattern); + 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) { @@ -192,14 +192,15 @@ TEST_F(TestFlightSqlClient, TestGetExported) { pb::sql::CommandGetExportedKeys command; command.set_catalog(catalog); - command.set_schema(schema); + 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)); - ASSERT_OK(sql_client_.GetExportedKeys(call_options_, &catalog, &schema, table)); + 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) { @@ -209,14 +210,15 @@ TEST_F(TestFlightSqlClient, TestGetImported) { pb::sql::CommandGetImportedKeys command; command.set_catalog(catalog); - command.set_schema(schema); + 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)); - ASSERT_OK(sql_client_.GetImportedKeys(call_options_, &catalog, &schema, table)); + 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) { @@ -226,14 +228,15 @@ TEST_F(TestFlightSqlClient, TestGetPrimary) { pb::sql::CommandGetPrimaryKeys command; command.set_catalog(catalog); - command.set_schema(schema); + 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)); - ASSERT_OK(sql_client_.GetPrimaryKeys(call_options_, &catalog, &schema, table)); + 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) { @@ -246,18 +249,21 @@ TEST_F(TestFlightSqlClient, TestGetCrossReference) { pb::sql::CommandGetCrossReference command; command.set_pk_catalog(pk_catalog); - command.set_pk_schema(pk_schema); + command.set_pk_db_schema(pk_schema); command.set_pk_table(pk_table); command.set_fk_catalog(fk_catalog); - command.set_fk_schema(fk_schema); + 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)); - ASSERT_OK(sql_client_.GetCrossReference(call_options_, &pk_catalog, &pk_schema, - pk_table, &fk_catalog, &fk_schema, fk_table)); + 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) { diff --git a/cpp/src/arrow/flight/sql/example/sqlite_server.cc b/cpp/src/arrow/flight/sql/example/sqlite_server.cc index de2bd5b7e02..a7159915769 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.cc @@ -62,8 +62,8 @@ std::string PrepareQueryForGetTables(const GetTables& command) { table_query << " and catalog_name='" << command.catalog.value() << "'"; } - if (command.schema_filter_pattern.has_value()) { - table_query << " and schema_name LIKE '" << command.schema_filter_pattern.value() + if (command.db_schema_filter_pattern.has_value()) { + table_query << " and schema_name LIKE '" << command.db_schema_filter_pattern.value() << "'"; } @@ -301,16 +301,16 @@ arrow::Result> SQLiteFlightSqlServer::DoGetCat } arrow::Result> SQLiteFlightSqlServer::GetFlightInfoSchemas( - const ServerCallContext& context, const GetSchemas& command, + const ServerCallContext& context, const GetDbSchemas& command, const FlightDescriptor& descriptor) { - return GetFlightInfoForCommand(descriptor, SqlSchema::GetSchemasSchema()); + return GetFlightInfoForCommand(descriptor, SqlSchema::GetDbSchemasSchema()); } -arrow::Result> SQLiteFlightSqlServer::DoGetSchemas( - const ServerCallContext& context, const GetSchemas& command) { +arrow::Result> SQLiteFlightSqlServer::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::GetSchemasSchema(); + const std::shared_ptr& schema = SqlSchema::GetDbSchemasSchema(); StringBuilder catalog_name_builder; ARROW_ASSIGN_OR_RAISE(auto catalog_name, catalog_name_builder.Finish()); @@ -524,15 +524,16 @@ arrow::Result> SQLiteFlightSqlServer::DoGetPri "table_name, type as table_type\n" "FROM sqlite_master) where 1=1 and pk != 0"; - if (command.catalog.has_value()) { - table_query << " and catalog_name LIKE '" << command.catalog.value() << "'"; + const TableRef& table_ref = command.table_ref; + if (table_ref.catalog.has_value()) { + table_query << " and catalog_name LIKE '" << table_ref.catalog.value() << "'"; } - if (command.schema.has_value()) { - table_query << " and schema_name LIKE '" << command.schema.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 '" << command.table << "'"; + table_query << " and table_name LIKE '" << table_ref.table << "'"; return DoGetSQLiteQuery(db_, table_query.str(), SqlSchema::GetPrimaryKeysSchema()); } @@ -579,12 +580,13 @@ SQLiteFlightSqlServer::GetFlightInfoImportedKeys(const ServerCallContext& contex arrow::Result> SQLiteFlightSqlServer::DoGetImportedKeys( const ServerCallContext& context, const GetImportedKeys& command) { - std::string filter = "fk_table_name = '" + command.table + "'"; - if (command.catalog.has_value()) { - filter += " AND fk_catalog_name = '" + command.catalog.value() + "'"; + 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 (command.schema.has_value()) { - filter += " AND fk_schema_name = '" + command.schema.value() + "'"; + if (table_ref.db_schema.has_value()) { + filter += " AND fk_schema_name = '" + table_ref.db_schema.value() + "'"; } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); @@ -600,12 +602,13 @@ SQLiteFlightSqlServer::GetFlightInfoExportedKeys(const ServerCallContext& contex arrow::Result> SQLiteFlightSqlServer::DoGetExportedKeys( const ServerCallContext& context, const GetExportedKeys& command) { - std::string filter = "pk_table_name = '" + command.table + "'"; - if (command.catalog.has_value()) { - filter += " AND pk_catalog_name = '" + command.catalog.value() + "'"; + 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 (command.schema.has_value()) { - filter += " AND pk_schema_name = '" + command.schema.value() + "'"; + if (table_ref.db_schema.has_value()) { + filter += " AND pk_schema_name = '" + table_ref.db_schema.value() + "'"; } std::string query = PrepareQueryForGetImportedOrExportedKeys(filter); @@ -622,20 +625,22 @@ SQLiteFlightSqlServer::GetFlightInfoCrossReference(const ServerCallContext& cont arrow::Result> SQLiteFlightSqlServer::DoGetCrossReference(const ServerCallContext& context, const GetCrossReference& command) { - std::string filter = "pk_table_name = '" + command.pk_table + "'"; - if (command.pk_catalog.has_value()) { - filter += " AND pk_catalog_name = '" + command.pk_catalog.value() + "'"; + 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 (command.pk_schema.has_value()) { - filter += " AND pk_schema_name = '" + command.pk_schema.value() + "'"; + if (pk_table_ref.db_schema.has_value()) { + filter += " AND pk_schema_name = '" + pk_table_ref.db_schema.value() + "'"; } - filter += " AND fk_table_name = '" + command.fk_table + "'"; - if (command.fk_catalog.has_value()) { - filter += " AND fk_catalog_name = '" + command.fk_catalog.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 (command.fk_schema.has_value()) { - filter += " AND fk_schema_name = '" + command.fk_schema.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); diff --git a/cpp/src/arrow/flight/sql/example/sqlite_server.h b/cpp/src/arrow/flight/sql/example/sqlite_server.h index f505c291e39..29548523257 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.h @@ -74,10 +74,10 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { arrow::Result> DoGetCatalogs( const ServerCallContext& context) override; arrow::Result> GetFlightInfoSchemas( - const ServerCallContext& context, const GetSchemas& command, + const ServerCallContext& context, const GetDbSchemas& command, const FlightDescriptor& descriptor) override; - arrow::Result> DoGetSchemas( - const ServerCallContext& context, const GetSchemas& command) override; + arrow::Result> DoGetDbSchemas( + const ServerCallContext& context, const GetDbSchemas& command) override; arrow::Result DoPutCommandStatementUpdate( const ServerCallContext& context, const StatementUpdate& update) override; arrow::Result CreatePreparedStatement( diff --git a/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc index 92007fa9eb3..569d7e8761d 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc @@ -18,7 +18,7 @@ #include "arrow/flight/sql/example/sqlite_sql_info.h" #include "arrow/flight/sql/FlightSql.pb.h" -#include "arrow/flight/sql/sql_info_types.h" +#include "arrow/flight/sql/types.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/sql/example/sqlite_sql_info.h b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.h index f6057540774..3c6dd42135e 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_sql_info.h +++ b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.h @@ -17,7 +17,7 @@ #pragma once -#include "arrow/flight/sql/sql_info_types.h" +#include "arrow/flight/sql/types.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/sql/server.cc b/cpp/src/arrow/flight/sql/server.cc index 6a34bdeb96a..6d328c07b0e 100644 --- a/cpp/src/arrow/flight/sql/server.cc +++ b/cpp/src/arrow/flight/sql/server.cc @@ -30,7 +30,7 @@ #include "arrow/util/checked_cast.h" #define PROPERTY_TO_OPTIONAL(COMMAND, PROPERTY) \ - COMMAND.has_##PROPERTY() ? util::make_optional(COMMAND.PROPERTY()) : util::nullopt; + COMMAND.has_##PROPERTY() ? util::make_optional(COMMAND.PROPERTY()) : util::nullopt namespace arrow { namespace flight { @@ -51,12 +51,10 @@ arrow::Result ParseCommandGetCrossReference( } GetCrossReference result; - result.pk_catalog = PROPERTY_TO_OPTIONAL(command, pk_catalog); - result.pk_schema = PROPERTY_TO_OPTIONAL(command, pk_schema); - result.pk_table = command.pk_table(); - result.fk_catalog = PROPERTY_TO_OPTIONAL(command, fk_catalog); - result.fk_schema = PROPERTY_TO_OPTIONAL(command, fk_schema); - result.fk_table = command.fk_table(); + 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; } @@ -68,9 +66,8 @@ arrow::Result ParseCommandGetImportedKeys( } GetImportedKeys result; - result.catalog = PROPERTY_TO_OPTIONAL(command, catalog); - result.schema = PROPERTY_TO_OPTIONAL(command, schema); - result.table = command.table(); + result.table_ref = {PROPERTY_TO_OPTIONAL(command, catalog), + PROPERTY_TO_OPTIONAL(command, db_schema), command.table()}; return result; } @@ -82,9 +79,8 @@ arrow::Result ParseCommandGetExportedKeys( } GetExportedKeys result; - result.catalog = PROPERTY_TO_OPTIONAL(command, catalog); - result.schema = PROPERTY_TO_OPTIONAL(command, schema); - result.table = command.table(); + result.table_ref = {PROPERTY_TO_OPTIONAL(command, catalog), + PROPERTY_TO_OPTIONAL(command, db_schema), command.table()}; return result; } @@ -96,9 +92,8 @@ arrow::Result ParseCommandGetPrimaryKeys( } GetPrimaryKeys result; - result.catalog = PROPERTY_TO_OPTIONAL(command, catalog); - result.schema = PROPERTY_TO_OPTIONAL(command, schema); - result.table = command.table(); + result.table_ref = {PROPERTY_TO_OPTIONAL(command, catalog), + PROPERTY_TO_OPTIONAL(command, db_schema), command.table()}; return result; } @@ -122,15 +117,16 @@ arrow::Result ParseCommandGetSqlInfo( return result; } -arrow::Result ParseCommandGetSchemas(const google::protobuf::Any& any) { - pb::sql::CommandGetSchemas command; +arrow::Result ParseCommandGetDbSchemas(const google::protobuf::Any& any) { + pb::sql::CommandGetDbSchemas command; if (!any.UnpackTo(&command)) { - return Status::Invalid("Unable to unpack CommandGetSchemas."); + return Status::Invalid("Unable to unpack CommandGetDbSchemas."); } - GetSchemas result; + GetDbSchemas result; result.catalog = PROPERTY_TO_OPTIONAL(command, catalog); - result.schema_filter_pattern = PROPERTY_TO_OPTIONAL(command, schema_filter_pattern); + result.db_schema_filter_pattern = + PROPERTY_TO_OPTIONAL(command, db_schema_filter_pattern); return result; } @@ -170,7 +166,8 @@ arrow::Result ParseCommandGetTables(const google::protobuf::Any& any) GetTables result; result.catalog = PROPERTY_TO_OPTIONAL(command, catalog); - result.schema_filter_pattern = PROPERTY_TO_OPTIONAL(command, schema_filter_pattern); + 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; @@ -279,8 +276,8 @@ Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context, } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(*info, GetFlightInfoCatalogs(context, request)); return Status::OK(); - } else if (any.Is()) { - ARROW_ASSIGN_OR_RAISE(GetSchemas internal_command, ParseCommandGetSchemas(any)); + } 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(); @@ -347,9 +344,9 @@ Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const Ticket } else if (any.Is()) { ARROW_ASSIGN_OR_RAISE(*stream, DoGetCatalogs(context)); return Status::OK(); - } else if (any.Is()) { - ARROW_ASSIGN_OR_RAISE(GetSchemas internal_command, ParseCommandGetSchemas(any)); - ARROW_ASSIGN_OR_RAISE(*stream, DoGetSchemas(context, internal_command)); + } 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)); @@ -585,14 +582,14 @@ arrow::Result> FlightSqlServerBase::DoGetSqlIn } arrow::Result> FlightSqlServerBase::GetFlightInfoSchemas( - const ServerCallContext& context, const GetSchemas& command, + const ServerCallContext& context, const GetDbSchemas& command, const FlightDescriptor& descriptor) { return Status::NotImplemented("GetFlightInfoSchemas not implemented"); } -arrow::Result> FlightSqlServerBase::DoGetSchemas( - const ServerCallContext& context, const GetSchemas& command) { - return Status::NotImplemented("DoGetSchemas not implemented"); +arrow::Result> FlightSqlServerBase::DoGetDbSchemas( + const ServerCallContext& context, const GetDbSchemas& command) { + return Status::NotImplemented("DoGetDbSchemas not implemented"); } arrow::Result> FlightSqlServerBase::GetFlightInfoTables( @@ -695,18 +692,18 @@ std::shared_ptr SqlSchema::GetCatalogsSchema() { return arrow::schema({field("catalog_name", utf8())}); } -std::shared_ptr SqlSchema::GetSchemasSchema() { +std::shared_ptr SqlSchema::GetDbSchemasSchema() { return arrow::schema( - {field("catalog_name", utf8()), field("schema_name", utf8(), false)}); + {field("catalog_name", utf8()), field("db_schema_name", utf8(), false)}); } std::shared_ptr SqlSchema::GetTablesSchema() { - return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8()), + 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("schema_name", utf8()), + return arrow::schema({field("catalog_name", utf8()), field("db_schema_name", utf8()), field("table_name", utf8()), field("table_type", utf8()), field("table_schema", binary())}); } @@ -716,16 +713,16 @@ std::shared_ptr SqlSchema::GetTableTypesSchema() { } std::shared_ptr SqlSchema::GetPrimaryKeysSchema() { - return arrow::schema({field("catalog_name", utf8()), field("schema_name", utf8()), + 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_schema_name", utf8(), true), + {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_schema_name", utf8(), true), + 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), diff --git a/cpp/src/arrow/flight/sql/server.h b/cpp/src/arrow/flight/sql/server.h index f63606e557a..b176de6e59e 100644 --- a/cpp/src/arrow/flight/sql/server.h +++ b/cpp/src/arrow/flight/sql/server.h @@ -26,7 +26,7 @@ #include "arrow/flight/server.h" #include "arrow/flight/sql/server.h" -#include "arrow/flight/sql/sql_info_types.h" +#include "arrow/flight/sql/types.h" #include "arrow/util/optional.h" namespace arrow { @@ -57,44 +57,34 @@ struct GetSqlInfo { std::vector info; }; -struct GetSchemas { +struct GetDbSchemas { util::optional catalog; - util::optional schema_filter_pattern; + util::optional db_schema_filter_pattern; }; struct GetTables { util::optional catalog; - util::optional schema_filter_pattern; + util::optional db_schema_filter_pattern; util::optional table_name_filter_pattern; std::vector table_types; bool include_schema; }; struct GetPrimaryKeys { - util::optional catalog; - util::optional schema; - std::string table; + TableRef table_ref; }; struct GetExportedKeys { - util::optional catalog; - util::optional schema; - std::string table; + TableRef table_ref; }; struct GetImportedKeys { - util::optional catalog; - util::optional schema; - std::string table; + TableRef table_ref; }; struct GetCrossReference { - util::optional pk_catalog; - util::optional pk_schema; - std::string pk_table; - util::optional fk_catalog; - util::optional fk_schema; - std::string fk_table; + TableRef pk_table_ref; + TableRef fk_table_ref; }; struct ActionCreatePreparedStatementRequest { @@ -218,21 +208,21 @@ class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase { /// \brief Get a FlightInfo for listing schemas. /// \param[in] context Per-call context. - /// \param[in] command The GetSchemas object which may contain filters for + /// \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 GetSchemas& command, + 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 GetSchemas object which may contain filters for + /// \param[in] command The GetDbSchemas object which may contain filters for /// catalog and schema name. /// \return The FlightDataStream containing the results. - virtual arrow::Result> DoGetSchemas( - const ServerCallContext& context, const GetSchemas& command); + virtual arrow::Result> DoGetDbSchemas( + const ServerCallContext& context, const GetDbSchemas& command); ///\brief Get a FlightInfo for listing tables. /// \param[in] context Per-call context. @@ -408,9 +398,9 @@ class ARROW_EXPORT SqlSchema { /// \return The default schema template. static std::shared_ptr GetCatalogsSchema(); - /// \brief Get the Schema used on GetSchemas response. + /// \brief Get the Schema used on GetDbSchemas response. /// \return The default schema template. - static std::shared_ptr GetSchemasSchema(); + static std::shared_ptr GetDbSchemasSchema(); /// \brief Get the Schema used on GetTables response when included schema /// flags is set to false. diff --git a/cpp/src/arrow/flight/sql/server_test.cc b/cpp/src/arrow/flight/sql/server_test.cc index 6db9548046c..45d3148cd2c 100644 --- a/cpp/src/arrow/flight/sql/server_test.cc +++ b/cpp/src/arrow/flight/sql/server_test.cc @@ -405,12 +405,12 @@ TEST_F(TestFlightSqlServer, TestCommandGetCatalogs) { ASSERT_EQ(0, table->num_rows()); } -TEST_F(TestFlightSqlServer, TestCommandGetSchemas) { +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->GetSchemas(options, catalog, schema_filter_pattern)); + sql_client->GetDbSchemas(options, catalog, schema_filter_pattern)); ASSERT_OK_AND_ASSIGN(auto stream, sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); @@ -418,7 +418,7 @@ TEST_F(TestFlightSqlServer, TestCommandGetSchemas) { std::shared_ptr
table; ASSERT_OK(stream->ReadAll(&table)); - const std::shared_ptr& expected_schema = SqlSchema::GetSchemasSchema(); + const std::shared_ptr& expected_schema = SqlSchema::GetDbSchemasSchema(); AssertSchemaEqual(expected_schema, table->schema()); ASSERT_EQ(0, table->num_rows()); @@ -605,11 +605,8 @@ TEST_F(TestFlightSqlServer, TestCommandPreparedStatementUpdate) { TEST_F(TestFlightSqlServer, TestCommandGetPrimaryKeys) { FlightCallOptions options = {}; - std::string* catalog = nullptr; - std::string* schema = nullptr; - std::string table_filter = "int%"; - ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetPrimaryKeys( - options, catalog, schema, table_filter)); + 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)); @@ -633,11 +630,8 @@ TEST_F(TestFlightSqlServer, TestCommandGetPrimaryKeys) { TEST_F(TestFlightSqlServer, TestCommandGetImportedKeys) { FlightCallOptions options = {}; - std::string* catalog = nullptr; - std::string* schema = nullptr; - std::string table_filter = "intTable"; - ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetImportedKeys( - options, catalog, schema, table_filter)); + 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)); @@ -669,11 +663,8 @@ TEST_F(TestFlightSqlServer, TestCommandGetImportedKeys) { TEST_F(TestFlightSqlServer, TestCommandGetExportedKeys) { FlightCallOptions options = {}; - std::string* catalog = nullptr; - std::string* schema = nullptr; - std::string table_filter = "foreignTable"; - ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetExportedKeys( - options, catalog, schema, table_filter)); + 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)); @@ -705,15 +696,10 @@ TEST_F(TestFlightSqlServer, TestCommandGetExportedKeys) { TEST_F(TestFlightSqlServer, TestCommandGetCrossReference) { FlightCallOptions options = {}; - std::string* pk_catalog = nullptr; - std::string* pk_schema = nullptr; - std::string pk_table = "foreignTable"; - std::string* fk_catalog = nullptr; - std::string* fk_schema = nullptr; - std::string fk_table = "intTable"; + 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_catalog, pk_schema, pk_table, - fk_catalog, fk_schema, fk_table)); + options, pk_table_ref, fk_table_ref)); ASSERT_OK_AND_ASSIGN(auto stream, sql_client->DoGet({}, flight_info->endpoints()[0].ticket)); diff --git a/cpp/src/arrow/flight/sql/sql_info_internal.h b/cpp/src/arrow/flight/sql/sql_info_internal.h index cb222e6f880..b18789c2549 100644 --- a/cpp/src/arrow/flight/sql/sql_info_internal.h +++ b/cpp/src/arrow/flight/sql/sql_info_internal.h @@ -17,7 +17,7 @@ #pragma once -#include "arrow/flight/sql/sql_info_types.h" +#include "arrow/flight/sql/types.h" namespace arrow { namespace flight { diff --git a/cpp/src/arrow/flight/sql/test_app_cli.cc b/cpp/src/arrow/flight/sql/test_app_cli.cc index 2f3000d9131..43c37bee2fe 100644 --- a/cpp/src/arrow/flight/sql/test_app_cli.cc +++ b/cpp/src/arrow/flight/sql/test_app_cli.cc @@ -29,6 +29,7 @@ #include "arrow/pretty_print.h" #include "arrow/status.h" #include "arrow/table.h" +#include "arrow/util/optional.h" using arrow::Result; using arrow::Schema; @@ -44,6 +45,7 @@ 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"); @@ -150,9 +152,9 @@ Status RunMain() { ARROW_RETURN_NOT_OK(prepared_statement->SetParameters(result)); ARROW_ASSIGN_OR_RAISE(info, prepared_statement->Execute()); - } else if (FLAGS_command == "GetSchemas") { + } else if (FLAGS_command == "GetDbSchemas") { ARROW_ASSIGN_OR_RAISE( - info, sql_client.GetSchemas(call_options, &FLAGS_catalog, &FLAGS_schema)); + 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") { @@ -160,14 +162,17 @@ Status RunMain() { info, sql_client.GetTables(call_options, &FLAGS_catalog, &FLAGS_schema, &FLAGS_table, false, nullptr)); } else if (FLAGS_command == "GetExportedKeys") { - ARROW_ASSIGN_OR_RAISE(info, sql_client.GetExportedKeys(call_options, &FLAGS_catalog, - &FLAGS_schema, FLAGS_table)); + 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") { - ARROW_ASSIGN_OR_RAISE(info, sql_client.GetImportedKeys(call_options, &FLAGS_catalog, - &FLAGS_schema, FLAGS_table)); + 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") { - ARROW_ASSIGN_OR_RAISE(info, sql_client.GetPrimaryKeys(call_options, &FLAGS_catalog, - &FLAGS_schema, FLAGS_table)); + 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, {})); } diff --git a/cpp/src/arrow/flight/sql/sql_info_types.h b/cpp/src/arrow/flight/sql/types.h similarity index 78% rename from cpp/src/arrow/flight/sql/sql_info_types.h rename to cpp/src/arrow/flight/sql/types.h index a8b87fc1143..edee161ba25 100644 --- a/cpp/src/arrow/flight/sql/sql_info_types.h +++ b/cpp/src/arrow/flight/sql/types.h @@ -21,17 +21,28 @@ #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 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/format/FlightSql.proto b/format/FlightSql.proto index 759654f49a7..495d429d93e 100644 --- a/format/FlightSql.proto +++ b/format/FlightSql.proto @@ -25,10 +25,10 @@ package arrow.flight.protocol.sql; /* * Represents a metadata request. Used in the command member of FlightDescriptor * for the following RPC calls: - * - GetSchema: return the schema of the query. + * - GetSchema: return the Arrow schema of the query. * - GetFlightInfo: execute the metadata request. * - * The returned schema will be: + * The returned Arrow schema will be: * < * info_name: uint32 not null, * value: dense_union< @@ -120,7 +120,7 @@ enum SqlInfo { SQL_DDL_TABLE = 502; /* - * Retrieves a uint32 value representing the enu uint32 ordinal for the case sensitivity of catalog, table and schema names. + * 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`. */ @@ -516,10 +516,10 @@ enum SqlInfo { */ 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 characters allowed in a schema name. + SQL_DB_SCHEMA_NAME_LENGTH = 552; - // Retrieves a uint32 value representing the maximum number of bytes allowed in a single row. + // 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. @@ -870,10 +870,10 @@ enum SqlSupportsConvert { /* * Represents a request to retrieve the list of catalogs on a Flight SQL enabled backend. * Used in the command member of FlightDescriptor for the following RPC calls: - * - GetSchema: return the schema of the query. + * - GetSchema: return the Arrow schema of the query. * - GetFlightInfo: execute the catalog metadata request. * - * The returned schema will be: + * The returned Arrow schema will be: * < * catalog_name: utf8 not null * > @@ -886,17 +886,17 @@ message CommandGetCatalogs { /* * Represents a request to retrieve the list of schemas on a Flight SQL enabled backend. * Used in the command member of FlightDescriptor for the following RPC calls: - * - GetSchema: return the schema of the query. + * - GetSchema: return the Arrow schema of the query. * - GetFlightInfo: execute the catalog metadata request. * - * The returned schema will be: + * The returned Arrow schema will be: * < * catalog_name: utf8, - * schema_name: utf8 not null + * db_schema_name: utf8 not null * > - * The returned data should be ordered by catalog_name, then schema_name. + * The returned data should be ordered by catalog_name, then db_schema_name. */ -message CommandGetSchemas { +message CommandGetDbSchemas { option (experimental) = true; /* @@ -908,30 +908,30 @@ message CommandGetSchemas { /* * Specifies a filter pattern for schemas to search for. - * When no schema_filter_pattern is provided, the pattern will not be used to narrow the search. + * 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 schema_filter_pattern = 2; + 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 schema of the query. + * - GetSchema: return the Arrow schema of the query. * - GetFlightInfo: execute the catalog metadata request. * - * The returned schema will be: + * The returned Arrow schema will be: * < * catalog_name: utf8, - * schema_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, schema_name, table_name, then table_type, followed by table_schema if requested. + * 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; @@ -945,12 +945,12 @@ message CommandGetTables { /* * Specifies a filter pattern for schemas to search for. - * When no schema_filter_pattern is provided, all schemas matching other filters are searched. + * 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 schema_filter_pattern = 2; + optional string db_schema_filter_pattern = 2; /* * Specifies a filter pattern for tables to search for. @@ -964,17 +964,17 @@ message CommandGetTables { // Specifies a filter of table types which must match. repeated string table_types = 4; - // Specifies if the schema should be returned for found tables. + // 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. * Used in the command member of FlightDescriptor for the following RPC calls: - * - GetSchema: return the schema of the query. + * - GetSchema: return the Arrow schema of the query. * - GetFlightInfo: execute the catalog metadata request. * - * The returned schema will be: + * The returned Arrow schema will be: * < * table_type: utf8 not null * > @@ -987,19 +987,19 @@ message CommandGetTableTypes { /* * Represents a request to retrieve the primary keys of a table on a Flight SQL enabled backend. * Used in the command member of FlightDescriptor for the following RPC calls: - * - GetSchema: return the schema of the query. + * - GetSchema: return the Arrow schema of the query. * - GetFlightInfo: execute the catalog metadata request. * - * The returned schema will be: + * The returned Arrow schema will be: * < * catalog_name: utf8, - * schema_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, schema_name, table_name, key_name, then key_sequence. + * The returned data should be ordered by catalog_name, db_schema_name, table_name, key_name, then key_sequence. */ message CommandGetPrimaryKeys { option (experimental) = true; @@ -1016,7 +1016,7 @@ message CommandGetPrimaryKeys { * An empty string retrieves those without a schema. * If omitted the schema name should not be used to narrow the search. */ - optional string schema = 2; + optional string db_schema = 2; // Specifies the table to get the primary keys for. string table = 3; @@ -1034,17 +1034,17 @@ enum UpdateDeleteRules { * 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 schema of the query. + * - GetSchema: return the Arrow schema of the query. * - GetFlightInfo: execute the catalog metadata request. * - * The returned schema will be: + * The returned Arrow schema will be: * < * pk_catalog_name: utf8, - * pk_schema_name: utf8, + * pk_db_schema_name: utf8, * pk_table_name: utf8 not null, * pk_column_name: utf8 not null, * fk_catalog_name: utf8, - * fk_schema_name: utf8, + * fk_db_schema_name: utf8, * fk_table_name: utf8 not null, * fk_column_name: utf8 not null, * key_sequence: int not null, @@ -1053,7 +1053,7 @@ enum UpdateDeleteRules { * update_rule: uint1 not null, * delete_rule: uint1 not null * > - * The returned data should be ordered by fk_catalog_name, fk_schema_name, fk_table_name, fk_key_name, then key_sequence. + * 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 { @@ -1071,7 +1071,7 @@ message CommandGetExportedKeys { * An empty string retrieves those without a schema. * If omitted the schema name should not be used to narrow the search. */ - optional string schema = 2; + optional string db_schema = 2; // Specifies the foreign key table to get the foreign keys for. string table = 3; @@ -1080,17 +1080,17 @@ message CommandGetExportedKeys { /* * 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 schema of the query. + * - GetSchema: return the Arrow schema of the query. * - GetFlightInfo: execute the catalog metadata request. * - * The returned schema will be: + * The returned Arrow schema will be: * < * pk_catalog_name: utf8, - * pk_schema_name: utf8, + * pk_db_schema_name: utf8, * pk_table_name: utf8 not null, * pk_column_name: utf8 not null, * fk_catalog_name: utf8, - * fk_schema_name: utf8, + * fk_db_schema_name: utf8, * fk_table_name: utf8 not null, * fk_column_name: utf8 not null, * key_sequence: int not null, @@ -1099,7 +1099,7 @@ message CommandGetExportedKeys { * update_rule: uint1 not null, * delete_rule: uint1 not null * > - * The returned data should be ordered by pk_catalog_name, pk_schema_name, pk_table_name, pk_key_name, then key_sequence. + * 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 @@ -1122,7 +1122,7 @@ message CommandGetImportedKeys { * An empty string retrieves those without a schema. * If omitted the schema name should not be used to narrow the search. */ - optional string schema = 2; + optional string db_schema = 2; // Specifies the primary key table to get the foreign keys for. string table = 3; @@ -1133,17 +1133,17 @@ message CommandGetImportedKeys { * 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 schema of the query. + * - GetSchema: return the Arrow schema of the query. * - GetFlightInfo: execute the catalog metadata request. * - * The returned schema will be: + * The returned Arrow schema will be: * < * pk_catalog_name: utf8, - * pk_schema_name: utf8, + * pk_db_schema_name: utf8, * pk_table_name: utf8 not null, * pk_column_name: utf8 not null, * fk_catalog_name: utf8, - * fk_schema_name: utf8, + * fk_db_schema_name: utf8, * fk_table_name: utf8 not null, * fk_column_name: utf8 not null, * key_sequence: int not null, @@ -1152,7 +1152,7 @@ message CommandGetImportedKeys { * update_rule: uint1 not null, * delete_rule: uint1 not null * > - * The returned data should be ordered by pk_catalog_name, pk_schema_name, pk_table_name, pk_key_name, then key_sequence. + * 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 @@ -1175,7 +1175,7 @@ message CommandGetCrossReference { * An empty string retrieves those without a schema. * If omitted the schema name should not be used to narrow the search. */ - optional string pk_schema = 2; + optional string pk_db_schema = 2; /** * The parent table name. It cannot be null. @@ -1194,7 +1194,7 @@ message CommandGetCrossReference { * An empty string retrieves those without a schema. * If omitted the schema name should not be used to narrow the search. */ - optional string fk_schema = 5; + optional string fk_db_schema = 5; /** * The foreign table name. It cannot be null. @@ -1253,7 +1253,7 @@ message ActionClosePreparedStatementRequest { /* * Represents a SQL query. Used in the command member of FlightDescriptor * for the following RPC calls: - * - GetSchema: return the schema of the query. + * - GetSchema: return the Arrow schema of the query. * - GetFlightInfo: execute the query. */ message CommandStatementQuery { From 2b3839aff39b474d77d515e0ce6290e8e1a9f81a Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Thu, 2 Dec 2021 13:48:49 -0300 Subject: [PATCH 227/237] Remove generated protobuf enum from example application --- .../flight/sql/example/sqlite_sql_info.cc | 46 +- cpp/src/arrow/flight/sql/types.h | 840 ++++++++++++++++++ 2 files changed, 862 insertions(+), 24 deletions(-) diff --git a/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc index 569d7e8761d..d3518a089e5 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc @@ -17,7 +17,6 @@ #include "arrow/flight/sql/example/sqlite_sql_info.h" -#include "arrow/flight/sql/FlightSql.pb.h" #include "arrow/flight/sql/types.h" namespace arrow { @@ -25,36 +24,35 @@ namespace flight { namespace sql { namespace example { -namespace flight_sql_pb = arrow::flight::protocol::sql; - /// \brief Gets the mapping from SQL info ids to SqlInfoResult instances. /// \return the cache. SqlInfoResultMap GetSqlInfoResultMap() { return { - {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_NAME, + {SqlInfoOptions::SqlInfo::FLIGHT_SQL_SERVER_NAME, SqlInfoResult(std::string("db_name"))}, - {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_VERSION, + {SqlInfoOptions::SqlInfo::FLIGHT_SQL_SERVER_VERSION, SqlInfoResult(std::string("sqlite 3"))}, - {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_ARROW_VERSION, + {SqlInfoOptions::SqlInfo::FLIGHT_SQL_SERVER_ARROW_VERSION, SqlInfoResult(std::string("7.0.0-SNAPSHOT" /* Only an example */))}, - {flight_sql_pb::SqlInfo::FLIGHT_SQL_SERVER_READ_ONLY, SqlInfoResult(false)}, - {flight_sql_pb::SqlInfo::SQL_DDL_CATALOG, + {SqlInfoOptions::SqlInfo::FLIGHT_SQL_SERVER_READ_ONLY, SqlInfoResult(false)}, + {SqlInfoOptions::SqlInfo::SQL_DDL_CATALOG, SqlInfoResult(false /* SQLite 3 does not support catalogs */)}, - {flight_sql_pb::SqlInfo::SQL_DDL_SCHEMA, + {SqlInfoOptions::SqlInfo::SQL_DDL_SCHEMA, SqlInfoResult(false /* SQLite 3 does not support schemas */)}, - {flight_sql_pb::SqlInfo::SQL_DDL_TABLE, SqlInfoResult(true)}, - {flight_sql_pb::SqlInfo::SQL_IDENTIFIER_CASE, - SqlInfoResult(int64_t(flight_sql_pb::SqlSupportedCaseSensitivity:: + {SqlInfoOptions::SqlInfo::SQL_DDL_TABLE, SqlInfoResult(true)}, + {SqlInfoOptions::SqlInfo::SQL_IDENTIFIER_CASE, + SqlInfoResult(int64_t(SqlInfoOptions::SqlSupportedCaseSensitivity:: SQL_CASE_SENSITIVITY_CASE_INSENSITIVE))}, - {flight_sql_pb::SqlInfo::SQL_IDENTIFIER_QUOTE_CHAR, + {SqlInfoOptions::SqlInfo::SQL_IDENTIFIER_QUOTE_CHAR, SqlInfoResult(std::string("\""))}, - {flight_sql_pb::SqlInfo::SQL_QUOTED_IDENTIFIER_CASE, - SqlInfoResult(int64_t(flight_sql_pb::SqlSupportedCaseSensitivity:: + {SqlInfoOptions::SqlInfo::SQL_QUOTED_IDENTIFIER_CASE, + SqlInfoResult(int64_t(SqlInfoOptions::SqlSupportedCaseSensitivity:: SQL_CASE_SENSITIVITY_CASE_INSENSITIVE))}, - {flight_sql_pb::SqlInfo::SQL_ALL_TABLES_ARE_SELECTABLE, SqlInfoResult(true)}, - {flight_sql_pb::SqlInfo::SQL_NULL_ORDERING, - SqlInfoResult(int64_t(flight_sql_pb::SqlNullOrdering::SQL_NULLS_SORTED_AT_START))}, - {flight_sql_pb::SqlInfo::SQL_KEYWORDS, + {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", @@ -202,21 +200,21 @@ SqlInfoResultMap GetSqlInfoResultMap() { "WINDOW", "WITH", "WITHOUT"}))}, - {flight_sql_pb::SqlInfo::SQL_NUMERIC_FUNCTIONS, + {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"}))}, - {flight_sql_pb::SqlInfo::SQL_STRING_FUNCTIONS, + {SqlInfoOptions::SqlInfo::SQL_STRING_FUNCTIONS, SqlInfoResult( std::vector({"SUBSTR", "TRIM", "LTRIM", "RTRIM", "LENGTH", "REPLACE", "UPPER", "LOWER", "INSTR"}))}, - {flight_sql_pb::SqlInfo::SQL_SUPPORTS_CONVERT, + {SqlInfoOptions::SqlInfo::SQL_SUPPORTS_CONVERT, SqlInfoResult(std::unordered_map>( - {{flight_sql_pb::SqlSupportsConvert::SQL_CONVERT_BIGINT, + {{SqlInfoOptions::SqlSupportsConvert::SQL_CONVERT_BIGINT, std::vector( - {flight_sql_pb::SqlSupportsConvert::SQL_CONVERT_INTEGER})}}))}}; + {SqlInfoOptions::SqlSupportsConvert::SQL_CONVERT_INTEGER})}}))}}; } } // namespace example diff --git a/cpp/src/arrow/flight/sql/types.h b/cpp/src/arrow/flight/sql/types.h index edee161ba25..3a227cc6a8e 100644 --- a/cpp/src/arrow/flight/sql/types.h +++ b/cpp/src/arrow/flight/sql/types.h @@ -36,6 +36,846 @@ using SqlInfoResult = /// \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; From 99ae0216466aee26944b31e10d46d2c3f372842d Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 3 Dec 2021 12:03:45 -0300 Subject: [PATCH 228/237] Remove need of RunServerInternal --- cpp/src/arrow/flight/sql/CMakeLists.txt | 7 ++++--- cpp/src/arrow/flight/sql/server_test.cc | 24 +++++++----------------- cpp/src/arrow/flight/sql/types.h | 2 ++ 3 files changed, 13 insertions(+), 20 deletions(-) diff --git a/cpp/src/arrow/flight/sql/CMakeLists.txt b/cpp/src/arrow/flight/sql/CMakeLists.txt index 178fec9f3e5..4a31f5ba2e2 100644 --- a/cpp/src/arrow/flight/sql/CMakeLists.txt +++ b/cpp/src/arrow/flight/sql/CMakeLists.txt @@ -70,7 +70,7 @@ endif() if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) find_package(SQLite3Alt REQUIRED) - set(flight_sql_test_server_SRCS + set(ARROW_FLIGHT_SQL_TEST_SERVER_SRCS example/sqlite_sql_info.cc example/sqlite_statement.cc example/sqlite_statement_batch_reader.cc @@ -81,14 +81,15 @@ if(ARROW_BUILD_TESTS OR ARROW_BUILD_EXAMPLES) SOURCES client_test.cc server_test.cc - ${flight_sql_test_server_SRCS} + ${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 ${flight_sql_test_server_SRCS}) + 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}) diff --git a/cpp/src/arrow/flight/sql/server_test.cc b/cpp/src/arrow/flight/sql/server_test.cc index 45d3148cd2c..f2a76f6bd5c 100644 --- a/cpp/src/arrow/flight/sql/server_test.cc +++ b/cpp/src/arrow/flight/sql/server_test.cc @@ -181,29 +181,19 @@ class TestFlightSqlServer : public ::testing::Test { std::condition_variable server_ready_cv; std::mutex server_ready_m; - Status RunServerInternal() { + static void RunServer(TestFlightSqlServer* fixture) { arrow::flight::Location location; - ARROW_CHECK_OK(arrow::flight::Location::ForGrpcUnix(unix_sock, &location)); + ARROW_CHECK_OK(arrow::flight::Location::ForGrpcUnix(fixture->unix_sock, &location)); arrow::flight::FlightServerOptions options(location); - ARROW_ASSIGN_OR_RAISE(server, - arrow::flight::sql::example::SQLiteFlightSqlServer::Create()) + ARROW_CHECK_OK(example::SQLiteFlightSqlServer::Create().Value(&fixture->server)); - ARROW_CHECK_OK(server->Init(options)); + ARROW_CHECK_OK(fixture->server->Init(options)); // Exit with a clean error code (0) on SIGTERM - ARROW_CHECK_OK(server->SetShutdownOnSignals({SIGTERM})); + ARROW_CHECK_OK(fixture->server->SetShutdownOnSignals({SIGTERM})); - server_ready_cv.notify_all(); - ARROW_CHECK_OK(server->Serve()); - - return arrow::Status::OK(); - } - - static void RunServer(TestFlightSqlServer* server) { - arrow::Status st = server->RunServerInternal(); - if (!st.ok()) { - std::cerr << st << std::endl; - } + fixture->server_ready_cv.notify_all(); + ARROW_CHECK_OK(fixture->server->Serve()); } }; diff --git a/cpp/src/arrow/flight/sql/types.h b/cpp/src/arrow/flight/sql/types.h index 3a227cc6a8e..44b8bca4718 100644 --- a/cpp/src/arrow/flight/sql/types.h +++ b/cpp/src/arrow/flight/sql/types.h @@ -17,6 +17,8 @@ #pragma once +#include +#include #include #include From ea94097953448a48ebd31215113859f00d06dc4a Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 3 Dec 2021 12:15:44 -0300 Subject: [PATCH 229/237] Use TCP instead on unix sockets on server_test.cc --- cpp/src/arrow/flight/sql/server_test.cc | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/cpp/src/arrow/flight/sql/server_test.cc b/cpp/src/arrow/flight/sql/server_test.cc index f2a76f6bd5c..bac7c3200f2 100644 --- a/cpp/src/arrow/flight/sql/server_test.cc +++ b/cpp/src/arrow/flight/sql/server_test.cc @@ -148,14 +148,14 @@ class TestFlightSqlServer : public ::testing::Test { protected: void SetUp() override { - unix_sock = std::tmpnam(nullptr); - server_thread.reset(new std::thread(RunServer, this)); + 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+unix://" << unix_sock; + ss << "grpc://localhost:" << port; std::string uri = ss.str(); std::unique_ptr client; @@ -175,25 +175,25 @@ class TestFlightSqlServer : public ::testing::Test { } private: - std::string unix_sock; + int port; std::shared_ptr server; std::unique_ptr server_thread; std::condition_variable server_ready_cv; std::mutex server_ready_m; - static void RunServer(TestFlightSqlServer* fixture) { + void RunServer() { arrow::flight::Location location; - ARROW_CHECK_OK(arrow::flight::Location::ForGrpcUnix(fixture->unix_sock, &location)); + ARROW_CHECK_OK(arrow::flight::Location::ForGrpcTcp("localhost", port, &location)); arrow::flight::FlightServerOptions options(location); - ARROW_CHECK_OK(example::SQLiteFlightSqlServer::Create().Value(&fixture->server)); + ARROW_CHECK_OK(example::SQLiteFlightSqlServer::Create().Value(&server)); - ARROW_CHECK_OK(fixture->server->Init(options)); + ARROW_CHECK_OK(server->Init(options)); // Exit with a clean error code (0) on SIGTERM - ARROW_CHECK_OK(fixture->server->SetShutdownOnSignals({SIGTERM})); + ARROW_CHECK_OK(server->SetShutdownOnSignals({SIGTERM})); - fixture->server_ready_cv.notify_all(); - ARROW_CHECK_OK(fixture->server->Serve()); + server_ready_cv.notify_all(); + ARROW_CHECK_OK(server->Serve()); } }; From de8600ca83046bdb51b1a6e1c6420e36896d7e3d Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 3 Dec 2021 13:49:55 -0300 Subject: [PATCH 230/237] nit: Fix indentation on cpp_build.sh --- ci/scripts/cpp_build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/scripts/cpp_build.sh b/ci/scripts/cpp_build.sh index cebf73f85e8..02718e57836 100755 --- a/ci/scripts/cpp_build.sh +++ b/ci/scripts/cpp_build.sh @@ -71,7 +71,7 @@ cmake \ -DARROW_FILESYSTEM=${ARROW_FILESYSTEM:-ON} \ -DARROW_FLIGHT=${ARROW_FLIGHT:-OFF} \ -DARROW_FLIGHT_SQL=${ARROW_FLIGHT_SQL:-OFF} \ - -DARROW_FUZZING=${ARROW_FUZZING:-OFF} \ + -DARROW_FUZZING=${ARROW_FUZZING:-OFF} \ -DARROW_GANDIVA_JAVA=${ARROW_GANDIVA_JAVA:-OFF} \ -DARROW_GANDIVA_PC_CXX_FLAGS=${ARROW_GANDIVA_PC_CXX_FLAGS:-} \ -DARROW_GANDIVA=${ARROW_GANDIVA:-OFF} \ From cfe9e2ac79412d375e1415f40445589fd33500d6 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 3 Dec 2021 16:27:46 -0300 Subject: [PATCH 231/237] Use EXPECT_RAISES_WITH_MESSAGE_THAT on TestFlightSqlServer#TestCommandGetSqlInfoNoInfo --- cpp/src/arrow/flight/sql/server_test.cc | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/flight/sql/server_test.cc b/cpp/src/arrow/flight/sql/server_test.cc index bac7c3200f2..8dfea7a013e 100644 --- a/cpp/src/arrow/flight/sql/server_test.cc +++ b/cpp/src/arrow/flight/sql/server_test.cc @@ -756,12 +756,10 @@ TEST_F(TestFlightSqlServer, TestCommandGetSqlInfo) { TEST_F(TestFlightSqlServer, TestCommandGetSqlInfoNoInfo) { FlightCallOptions call_options; ASSERT_OK_AND_ASSIGN(auto flight_info, sql_client->GetSqlInfo(call_options, {999999})); - auto result = sql_client->DoGet(call_options, flight_info->endpoints()[0].ticket); - ASSERT_FALSE(result.ok()); - ASSERT_TRUE(result.status().IsKeyError()); - ASSERT_THAT(result.status().message(), - ::testing::HasSubstr("No information for SQL info number 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 From 6a928ca82ae69e579d4785c092d069f6b2439ceb Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 3 Dec 2021 17:09:25 -0300 Subject: [PATCH 232/237] Move implementation of methods and data from SQLiteFlightSqlServer to SQLiteFlightSqlServer::Impl and remove dependency of boost-uuid (#221) * Use pimpl idiom on sqlite_server and add comments on protobuf file * Correctly implement impl pattern --- .../arrow/flight/sql/example/sqlite_server.cc | 809 +++++++++++------- .../arrow/flight/sql/example/sqlite_server.h | 12 +- cpp/src/arrow/flight/sql/server.h | 3 +- cpp/vcpkg.json | 1 - format/FlightSql.proto | 7 +- 5 files changed, 497 insertions(+), 335 deletions(-) diff --git a/cpp/src/arrow/flight/sql/example/sqlite_server.cc b/cpp/src/arrow/flight/sql/example/sqlite_server.cc index a7159915769..7e0edd1109f 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.cc @@ -21,7 +21,8 @@ #include #include -#include +#include +#include #include "arrow/api.h" #include "arrow/flight/sql/example/sqlite_sql_info.h" @@ -39,12 +40,9 @@ namespace { /// \brief Gets a SqliteStatement by given handle arrow::Result> GetStatementByHandle( - const std::map>& - prepared_statements, + const std::map>& prepared_statements, const std::string& handle) { - const auto& uuid = boost::lexical_cast(handle); - - auto search = prepared_statements.find(uuid); + auto search = prepared_statements.find(handle); if (search == prepared_statements.end()) { return Status::Invalid("Prepared statement not found"); } @@ -140,6 +138,60 @@ Status SetParametersOnSQLiteStatement(sqlite3_stmt* stmt, FlightMessageReader* r 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) { @@ -163,9 +215,405 @@ std::shared_ptr GetArrowType(const char* sqlite_type) { } } -SQLiteFlightSqlServer::SQLiteFlightSqlServer(sqlite3* db) : db_(db), uuid_generator_({}) { - assert(db_); -} +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 GenerateString() { + uint32_t length = 16; + // Character set used to generate random string + const std::string charset = + "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz"; + + std::uniform_int_distribution dist(0, static_cast(charset.size() - 1)); + std::string ret(length, 'x'); + + for (uint32_t i = 0; i < length; i++) { + ret[i] = charset[dist(gen_)]; + } + 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 = GenerateString(); + 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; @@ -182,7 +630,9 @@ arrow::Result> SQLiteFlightSqlServer::Cre return Status::Invalid(err_msg); } - std::shared_ptr result(new SQLiteFlightSqlServer(db)); + 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); } @@ -211,440 +661,155 @@ arrow::Result> SQLiteFlightSqlServer::Cre return result; } -SQLiteFlightSqlServer::~SQLiteFlightSqlServer() { sqlite3_close(db_); } +SQLiteFlightSqlServer::~SQLiteFlightSqlServer() = default; Status SQLiteFlightSqlServer::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(); -} - -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)); + return impl_->ExecuteSql(sql); } arrow::Result> SQLiteFlightSqlServer::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)); + return impl_->GetFlightInfoStatement(context, command, descriptor); } arrow::Result> SQLiteFlightSqlServer::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)); + return impl_->DoGetStatement(context, command); } arrow::Result> SQLiteFlightSqlServer::GetFlightInfoCatalogs( const ServerCallContext& context, const FlightDescriptor& descriptor) { - return GetFlightInfoForCommand(descriptor, SqlSchema::GetCatalogsSchema()); + return impl_->GetFlightInfoCatalogs(context, descriptor); } arrow::Result> SQLiteFlightSqlServer::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)); + return impl_->DoGetCatalogs(context); } arrow::Result> SQLiteFlightSqlServer::GetFlightInfoSchemas( const ServerCallContext& context, const GetDbSchemas& command, const FlightDescriptor& descriptor) { - return GetFlightInfoForCommand(descriptor, SqlSchema::GetDbSchemasSchema()); + return impl_->GetFlightInfoSchemas(context, command, descriptor); } arrow::Result> SQLiteFlightSqlServer::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)); + return impl_->DoGetDbSchemas(context, command); } arrow::Result> SQLiteFlightSqlServer::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)); + return impl_->GetFlightInfoTables(context, command, descriptor); } arrow::Result> SQLiteFlightSqlServer::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)); - } + return impl_->DoGetTables(context, command); } arrow::Result SQLiteFlightSqlServer::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(); + return impl_->DoPutCommandStatementUpdate(context, command); } arrow::Result SQLiteFlightSqlServer::CreatePreparedStatement( const ServerCallContext& context, const ActionCreatePreparedStatementRequest& request) { - std::shared_ptr statement; - ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, request.query)); - boost::uuids::uuid uuid = uuid_generator_(); - prepared_statements_[uuid] = 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 = boost::uuids::to_string(uuid)}; - - return result; + return impl_->CreatePreparedStatement(context, request); } Status SQLiteFlightSqlServer::ClosePreparedStatement( const ServerCallContext& context, const ActionClosePreparedStatementRequest& request) { - const std::string& prepared_statement_handle = request.prepared_statement_handle; - const auto& uuid = boost::lexical_cast(prepared_statement_handle); - - auto search = prepared_statements_.find(uuid); - if (search != prepared_statements_.end()) { - prepared_statements_.erase(uuid); - } else { - return Status::Invalid("Prepared statement not found"); - } - - return Status::OK(); + return impl_->ClosePreparedStatement(context, request); } arrow::Result> SQLiteFlightSqlServer::GetFlightInfoPreparedStatement( const ServerCallContext& context, const PreparedStatementQuery& command, const FlightDescriptor& descriptor) { - const std::string& prepared_statement_handle = command.prepared_statement_handle; - const auto& uuid = boost::lexical_cast(prepared_statement_handle); - - auto search = prepared_statements_.find(uuid); - 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); + return impl_->GetFlightInfoPreparedStatement(context, command, descriptor); } arrow::Result> SQLiteFlightSqlServer::DoGetPreparedStatement(const ServerCallContext& context, const PreparedStatementQuery& command) { - const std::string& prepared_statement_handle = command.prepared_statement_handle; - const auto& uuid = boost::lexical_cast(prepared_statement_handle); - - auto search = prepared_statements_.find(uuid); - 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)); + return impl_->DoGetPreparedStatement(context, command); } Status SQLiteFlightSqlServer::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(); + return impl_->DoPutPreparedStatementQuery(context, command, reader, writer); } arrow::Result SQLiteFlightSqlServer::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(); + return impl_->DoPutPreparedStatementUpdate(context, command, reader); } arrow::Result> SQLiteFlightSqlServer::GetFlightInfoTableTypes( const ServerCallContext& context, const FlightDescriptor& descriptor) { - return GetFlightInfoForCommand(descriptor, SqlSchema::GetTableTypesSchema()); + return impl_->GetFlightInfoTableTypes(context, descriptor); } arrow::Result> SQLiteFlightSqlServer::DoGetTableTypes( const ServerCallContext& context) { - std::string query = "SELECT DISTINCT type as table_type FROM sqlite_master"; - - return DoGetSQLiteQuery(db_, query, SqlSchema::GetTableTypesSchema()); + return impl_->DoGetTableTypes(context); } arrow::Result> SQLiteFlightSqlServer::GetFlightInfoPrimaryKeys(const ServerCallContext& context, const GetPrimaryKeys& command, const FlightDescriptor& descriptor) { - return GetFlightInfoForCommand(descriptor, SqlSchema::GetPrimaryKeysSchema()); + return impl_->GetFlightInfoPrimaryKeys(context, command, descriptor); } arrow::Result> SQLiteFlightSqlServer::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()); -} - -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)"; + return impl_->DoGetPrimaryKeys(context, command); } arrow::Result> SQLiteFlightSqlServer::GetFlightInfoImportedKeys(const ServerCallContext& context, const GetImportedKeys& command, const FlightDescriptor& descriptor) { - return GetFlightInfoForCommand(descriptor, SqlSchema::GetImportedKeysSchema()); + return impl_->GetFlightInfoImportedKeys(context, command, descriptor); } arrow::Result> SQLiteFlightSqlServer::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()); + return impl_->DoGetImportedKeys(context, command); } arrow::Result> SQLiteFlightSqlServer::GetFlightInfoExportedKeys(const ServerCallContext& context, const GetExportedKeys& command, const FlightDescriptor& descriptor) { - return GetFlightInfoForCommand(descriptor, SqlSchema::GetExportedKeysSchema()); + return impl_->GetFlightInfoExportedKeys(context, command, descriptor); } arrow::Result> SQLiteFlightSqlServer::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()); + return impl_->DoGetExportedKeys(context, command); } arrow::Result> SQLiteFlightSqlServer::GetFlightInfoCrossReference(const ServerCallContext& context, const GetCrossReference& command, const FlightDescriptor& descriptor) { - return GetFlightInfoForCommand(descriptor, SqlSchema::GetCrossReferenceSchema()); + return impl_->GetFlightInfoCrossReference(context, command, descriptor); } arrow::Result> SQLiteFlightSqlServer::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()); + return impl_->DoGetCrossReference(context, command); } } // namespace example diff --git a/cpp/src/arrow/flight/sql/example/sqlite_server.h b/cpp/src/arrow/flight/sql/example/sqlite_server.h index 29548523257..b2954b8703e 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_server.h +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.h @@ -19,9 +19,6 @@ #include -#include -#include -#include #include #include @@ -133,13 +130,10 @@ class SQLiteFlightSqlServer : public FlightSqlServerBase { const ServerCallContext& context, const GetPrimaryKeys& command) override; private: - sqlite3* db_; - boost::uuids::random_generator uuid_generator_; - std::map> prepared_statements_; + class Impl; + std::shared_ptr impl_; - /// SQLiteFlightSqlServer - /// \param db The db parameter from SQLite. The Server it is taking the ownership. - explicit SQLiteFlightSqlServer(sqlite3* db); + explicit SQLiteFlightSqlServer(std::shared_ptr impl); }; } // namespace example diff --git a/cpp/src/arrow/flight/sql/server.h b/cpp/src/arrow/flight/sql/server.h index b176de6e59e..1d6101683c1 100644 --- a/cpp/src/arrow/flight/sql/server.h +++ b/cpp/src/arrow/flight/sql/server.h @@ -101,7 +101,8 @@ struct ActionCreatePreparedStatementResult { std::string prepared_statement_handle; }; -/// \brief A utility function to create a ticket for a statement query. +/// \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. diff --git a/cpp/vcpkg.json b/cpp/vcpkg.json index 11ff74bf841..d9505b67383 100644 --- a/cpp/vcpkg.json +++ b/cpp/vcpkg.json @@ -19,7 +19,6 @@ "boost-multiprecision", "boost-process", "boost-system", - "boost-uuid", "brotli", "bzip2", "c-ares", diff --git a/format/FlightSql.proto b/format/FlightSql.proto index 495d429d93e..25413d255bd 100644 --- a/format/FlightSql.proto +++ b/format/FlightSql.proto @@ -869,6 +869,7 @@ enum SqlSupportsConvert { /* * 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. @@ -884,7 +885,8 @@ message CommandGetCatalogs { } /* - * Represents a request to retrieve the list of schemas on a Flight SQL enabled backend. + * 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. @@ -970,6 +972,7 @@ message CommandGetTables { /* * 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. * 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. @@ -1265,7 +1268,7 @@ message CommandStatementQuery { /** * Represents a ticket resulting from GetFlightInfo with a CommandStatementQuery. - * This should be treated as an opaque value, that is, clients should not attempt to parse this. + * 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; From 023f71a12fbd233dbba2571c7935db454516293e Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Fri, 3 Dec 2021 16:47:34 -0300 Subject: [PATCH 233/237] Use std::generate_n to generate random string on sqlite_server.cc --- .../arrow/flight/sql/example/sqlite_server.cc | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/cpp/src/arrow/flight/sql/example/sqlite_server.cc b/cpp/src/arrow/flight/sql/example/sqlite_server.cc index 7e0edd1109f..470853d947c 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.cc @@ -225,18 +225,13 @@ class SQLiteFlightSqlServer::Impl { ~Impl() { sqlite3_close(db_); } - std::string GenerateString() { + std::string GenerateRandomString() { uint32_t length = 16; - // Character set used to generate random string - const std::string charset = - "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz"; - std::uniform_int_distribution dist(0, static_cast(charset.size() - 1)); - std::string ret(length, 'x'); - - for (uint32_t i = 0; i < length; i++) { - ret[i] = charset[dist(gen_)]; - } + 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; } @@ -368,7 +363,7 @@ class SQLiteFlightSqlServer::Impl { const ActionCreatePreparedStatementRequest& request) { std::shared_ptr statement; ARROW_ASSIGN_OR_RAISE(statement, SqliteStatement::Create(db_, request.query)); - const std::string handle = GenerateString(); + const std::string handle = GenerateRandomString(); prepared_statements_[handle] = statement; ARROW_ASSIGN_OR_RAISE(auto dataset_schema, statement->GetSchema()); From dd9d507997e1bcf88aeb3511889dcb3f6b777283 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 6 Dec 2021 15:00:19 -0300 Subject: [PATCH 234/237] Remove dependency on boost/lexical_cast.hpp --- cpp/src/arrow/flight/sql/example/sqlite_server.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/flight/sql/example/sqlite_server.cc b/cpp/src/arrow/flight/sql/example/sqlite_server.cc index 470853d947c..dde364f64e3 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_server.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_server.cc @@ -20,9 +20,9 @@ #include #include -#include #include #include +#include #include "arrow/api.h" #include "arrow/flight/sql/example/sqlite_sql_info.h" From 9a9b536acf207456c8050d165c4f1a12c7d71010 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 6 Dec 2021 15:01:21 -0300 Subject: [PATCH 235/237] Change SQL_NUMERIC_FUNCTIONS result on sqlite_sql_info to uppercase --- cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc index d3518a089e5..94f25b39017 100644 --- a/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc +++ b/cpp/src/arrow/flight/sql/example/sqlite_sql_info.cc @@ -202,10 +202,10 @@ SqlInfoResultMap GetSqlInfoResultMap() { "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"}))}, + {"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", From 076187ec3aa18295c92de1f38b9036e66fa8ca7e Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 6 Dec 2021 15:02:45 -0300 Subject: [PATCH 236/237] Add better description to table types on FlightSql.proto --- format/FlightSql.proto | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/format/FlightSql.proto b/format/FlightSql.proto index 25413d255bd..d3f96f9c390 100644 --- a/format/FlightSql.proto +++ b/format/FlightSql.proto @@ -963,7 +963,11 @@ message CommandGetTables { */ optional string table_name_filter_pattern = 3; - // Specifies a filter of table types which must match. + /* + * 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. @@ -973,6 +977,7 @@ message CommandGetTables { /* * 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. From 72ce72ba855909052f7dfb898105b419697157c8 Mon Sep 17 00:00:00 2001 From: Rafael Telles Date: Mon, 6 Dec 2021 16:55:20 -0300 Subject: [PATCH 237/237] Fix documentation for GetSqlInfo on FlightSql.proto --- format/FlightSql.proto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/format/FlightSql.proto b/format/FlightSql.proto index d3f96f9c390..23ada5c6e48 100644 --- a/format/FlightSql.proto +++ b/format/FlightSql.proto @@ -441,7 +441,7 @@ enum SqlInfo { * - 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 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];