lidavidm commented on a change in pull request #11507:
URL: https://github.com/apache/arrow/pull/11507#discussion_r738781330



##########
File path: cpp/src/arrow/flight/flight-sql/CMakeLists.txt
##########
@@ -0,0 +1,116 @@
+# 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")

Review comment:
       nit: just to point this out again, I think we should stick with 
underscores in directory names to be consistent.

##########
File path: cpp/src/arrow/flight/flight-sql/CMakeLists.txt
##########
@@ -0,0 +1,116 @@
+# 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")
+
+set(PROTO_DEPENDS ${FLIGHT_SQL_PROTO} ${ARROW_PROTOBUF_LIBPROTOBUF})
+
+add_custom_command(OUTPUT ${FLIGHT_SQL_GENERATED_PROTO_FILES}
+                   COMMAND ${ARROW_PROTOBUF_PROTOC} 
"-I${FLIGHT_SQL_PROTO_PATH}"
+                           "--cpp_out=${CMAKE_CURRENT_BINARY_DIR}" 
"${FLIGHT_SQL_PROTO}"
+                   DEPENDS ${PROTO_DEPENDS})
+
+set_source_files_properties(${FLIGHT_SQL_GENERATED_PROTO_FILES} PROPERTIES 
GENERATED TRUE)
+
+add_custom_target(flight_sql_protobuf_gen ALL DEPENDS 
${FLIGHT_SQL_GENERATED_PROTO_FILES})
+
+set(ARROW_FLIGHT_SQL_SRCS server.cc client.cc client_internal.cc)
+
+add_arrow_lib(arrow_flight_sql
+              CMAKE_PACKAGE_NAME
+              ArrowFlightSql
+              PKG_CONFIG_NAME
+              arrow-flight-sql
+              OUTPUTS
+              ARROW_FLIGHT_SQL_LIBRARIES
+              SOURCES
+              ${ARROW_FLIGHT_SQL_SRCS}
+              PRECOMPILED_HEADERS
+              "$<$<COMPILE_LANGUAGE:CXX>:arrow/flight/flight-sql/pch.h>"
+              DEPENDENCIES
+              flight_sql_protobuf_gen
+              SHARED_LINK_FLAGS
+              ${ARROW_VERSION_SCRIPT_FLAGS} # Defined in 
cpp/arrow/CMakeLists.txt
+              SHARED_LINK_LIBS
+              arrow_flight_shared
+              ${ARROW_FLIGHT_SQL_LINK_LIBS}

Review comment:
       This variable doesn't appear to ever be set - is it needed?

##########
File path: cpp/src/arrow/flight/flight-sql/server.h
##########
@@ -0,0 +1,493 @@
+// 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 <arrow/flight/server.h>
+#include <sqlite3.h>
+
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+
+#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 {
+  std::string statement_handle;
+};
+
+using PreparedStatementQuery = struct PreparedStatementQuery {
+  std::string prepared_statement_handle;
+};
+
+using PreparedStatementUpdate = struct PreparedStatementUpdate {
+  std::string prepared_statement_handle;
+};
+
+using GetSqlInfo = struct GetSqlInfo {
+  // TODO: To be implemented.
+};
+
+using GetSchemas = struct GetSchemas {
+  bool has_catalog;
+  std::string catalog;
+  bool has_schema_filter_pattern;
+  std::string schema_filter_pattern;
+};
+
+using GetTables = 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;
+  std::vector<std::string> table_types;
+  bool include_schema;
+};
+
+using GetPrimaryKeys = struct GetPrimaryKeys {
+  bool has_catalog;
+  std::string catalog;
+  bool has_schema;
+  std::string schema;
+  std::string table;
+};
+
+using GetExportedKeys = struct GetExportedKeys {
+  bool has_catalog;
+  std::string catalog;
+  bool has_schema;
+  std::string schema;
+  std::string table;
+};
+
+using GetImportedKeys = struct GetImportedKeys {
+  bool has_catalog;
+  std::string catalog;
+  bool has_schema;
+  std::string schema;
+  std::string table;
+};
+
+using GetCrossReference = struct GetCrossReference {
+  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 {
+  std::string query;
+};
+
+using ActionClosePreparedStatementRequest = struct 
ActionClosePreparedStatementRequest {
+  std::string prepared_statement_handle;
+};
+
+using ActionCreatePreparedStatementResult = struct 
ActionCreatePreparedStatementResult {
+  std::shared_ptr<Schema> dataset_schema;
+  std::shared_ptr<Schema> parameter_schema;
+  std::string prepared_statement_handle;
+};
+
+class FlightSqlServerBase : public FlightServerBase {

Review comment:
       ```suggestion
   class ARROW_EXPORT FlightSqlServerBase : public FlightServerBase {
   ```

##########
File path: cpp/src/arrow/flight/flight-sql/server.h
##########
@@ -0,0 +1,493 @@
+// 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 <arrow/flight/server.h>
+#include <sqlite3.h>
+
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>

Review comment:
       Ditto for boost.

##########
File path: cpp/src/arrow/flight/flight-sql/server_test.cc
##########
@@ -0,0 +1,653 @@
+// 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/api.h>
+#include <arrow/flight/api.h>
+#include <arrow/flight/flight-sql/api.h>
+#include <arrow/flight/flight-sql/example/sqlite_server.h>
+#include <arrow/flight/flight-sql/server.h>
+#include <arrow/flight/test_util.h>
+#include <arrow/flight/types.h>
+#include <arrow/testing/gtest_util.h>
+#include <gmock/gmock.h>
+#include <google/protobuf/any.pb.h>
+#include <gtest/gtest.h>
+
+#define unparen(...) __VA_ARGS__
+#define DECLARE_ARRAY(ARRAY_NAME, TYPE_CLASS, DATA)     \
+  std::shared_ptr<arrow::TYPE_CLASS##Array> 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<arrow::BinaryArray> 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<arrow::TYPE_CLASS##Array> 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;
+
+namespace pb = arrow::flight::protocol;
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+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<FlightClient> 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);

Review comment:
       Ah sorry if it wasn't clear - we need to do delete or use unique_ptr, 
since AddressSanitizer-enabled builds will fail here since we're pairing new 
and free.

##########
File path: cpp/src/arrow/flight/flight-sql/client.cc
##########
@@ -0,0 +1,442 @@
+// 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/buffer.h>
+#include <arrow/flight/flight-sql/FlightSql.pb.h>
+#include <arrow/flight/flight-sql/client.h>
+#include <arrow/flight/flight-sql/client_internal.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>
+#include <google/protobuf/any.pb.h>
+
+#include <memory>
+#include <utility>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+FlightSqlClient::FlightSqlClient(std::shared_ptr<internal::FlightClientImpl> 
client)
+    : impl_(std::move(client)) {}
+
+FlightSqlClient::FlightSqlClient(std::unique_ptr<FlightClient> client)
+    : impl_(internal::FlightClientImpl_Create(std::move(client))) {}
+
+FlightSqlClient::PreparedStatement::PreparedStatement(
+    std::shared_ptr<internal::FlightClientImpl> 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;
+
+FlightSqlClient::PreparedStatement::~PreparedStatement() {
+  const Status status = Close();
+  if (!status.ok()) {
+    ARROW_LOG(ERROR) << "Failed to delete PreparedStatement: " << 
status.ToString();
+  }
+}
+
+inline FlightDescriptor GetFlightDescriptorForCommand(
+    const google::protobuf::Message& command) {
+  google::protobuf::Any any;
+  any.PackFrom(command);
+
+  const std::string& string = any.SerializeAsString();
+  return FlightDescriptor::Command(string);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> GetFlightInfoForCommand(
+    internal::FlightClientImpl& client, const FlightCallOptions& options,
+    const google::protobuf::Message& command) {
+  const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command);
+
+  std::unique_ptr<FlightInfo> flight_info;
+  ARROW_RETURN_NOT_OK(internal::FlightClientImpl_GetFlightInfo(client, options,
+                                                               descriptor, 
&flight_info));
+
+  return std::move(flight_info);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::Execute(
+    const FlightCallOptions& options, const std::string& query) const {
+  pb::sql::CommandStatementQuery command;
+  command.set_query(query);
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<int64_t> FlightSqlClient::ExecuteUpdate(const FlightCallOptions& 
options,
+                                                      const std::string& 
query) const {
+  pb::sql::CommandStatementUpdate command;
+  command.set_query(query);
+
+  const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command);
+
+  std::unique_ptr<FlightStreamWriter> writer;
+  std::unique_ptr<FlightMetadataReader> reader;
+
+  ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoPut(*impl_, options, 
descriptor,
+                                                       NULLPTR, &writer, 
&reader));
+
+  std::shared_ptr<Buffer> metadata;
+
+  ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata));
+
+  pb::sql::DoPutUpdateResult doPutUpdateResult;
+
+  pb::sql::DoPutUpdateResult result;
+  if (!result.ParseFromArray(metadata->data(), 
static_cast<int>(metadata->size()))) {
+    return Status::Invalid("Unable to parse DoPutUpdateResult object.");
+  }
+
+  return result.record_count();
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::GetCatalogs(
+    const FlightCallOptions& options) const {
+  pb::sql::CommandGetCatalogs command;
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::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);
+  }
+  if (schema_filter_pattern != NULLPTR) {
+    command.set_schema_filter_pattern(*schema_filter_pattern);
+  }
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> 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<std::string>& table_types) const {
+  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(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::GetPrimaryKeys(
+    const FlightCallOptions& options, const std::string* catalog,
+    const std::string* schema, const std::string& table) const {
+  pb::sql::CommandGetPrimaryKeys command;
+
+  if (catalog != NULLPTR) {
+    command.set_catalog(*catalog);
+  }
+
+  if (schema != NULLPTR) {
+    command.set_schema(*schema);
+  }
+
+  command.set_table(table);
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::GetExportedKeys(
+    const FlightCallOptions& options, const std::string* catalog,
+    const std::string* schema, const std::string& table) const {
+  pb::sql::CommandGetExportedKeys command;
+
+  if (catalog != NULLPTR) {
+    command.set_catalog(*catalog);
+  }
+
+  if (schema != NULLPTR) {
+    command.set_schema(*schema);
+  }
+
+  command.set_table(table);
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::GetImportedKeys(
+    const FlightCallOptions& options, const std::string* catalog,
+    const std::string* schema, const std::string& table) const {
+  pb::sql::CommandGetImportedKeys command;
+
+  if (catalog != NULLPTR) {
+    command.set_catalog(*catalog);
+  }
+
+  if (schema != NULLPTR) {
+    command.set_schema(*schema);
+  }
+
+  command.set_table(table);
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> 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 {
+  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(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::GetTableTypes(
+    const FlightCallOptions& options) const {
+  pb::sql::CommandGetTableTypes command;
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightStreamReader>> FlightSqlClient::DoGet(
+    const FlightCallOptions& options, const Ticket& ticket) const {
+  std::unique_ptr<FlightStreamReader> stream;
+  ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoGet(*impl_, options, 
ticket, &stream));
+
+  return std::move(stream);
+}
+
+arrow::Result<std::shared_ptr<FlightSqlClient::PreparedStatement>>
+FlightSqlClient::Prepare(const FlightCallOptions& options, const std::string& 
query) {
+  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<ResultStream> results;
+
+  ARROW_RETURN_NOT_OK(
+      internal::FlightClientImpl_DoAction(*impl_, options, action, &results));
+
+  std::unique_ptr<Result> result;
+  ARROW_RETURN_NOT_OK(results->Next(&result));
+
+  google::protobuf::Any prepared_result;
+
+  std::shared_ptr<Buffer> message = std::move(result->body);
+  if (!prepared_result.ParseFromArray(message->data(),
+                                      static_cast<int>(message->size()))) {
+    return Status::Invalid("Unable to parse packed 
ActionCreatePreparedStatementResult");
+  }
+
+  pb::sql::ActionCreatePreparedStatementResult prepared_statement_result;
+
+  if (!prepared_result.UnpackTo(&prepared_statement_result)) {
+    return Status::Invalid("Unable to unpack 
ActionCreatePreparedStatementResult");
+  }
+
+  return std::make_shared<PreparedStatement>(impl_, query, 
prepared_statement_result,
+                                             options);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> 
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());
+
+  google::protobuf::Any any;
+  any.PackFrom(execute_query_command);
+
+  const std::string& string = any.SerializeAsString();
+  const FlightDescriptor descriptor = FlightDescriptor::Command(string);
+
+  if (parameter_binding_ && parameter_binding_->num_rows() > 0) {
+    std::unique_ptr<FlightStreamWriter> writer;
+    std::unique_ptr<FlightMetadataReader> 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->DoneWriting());
+    // Wait for the server to ack the result
+    std::shared_ptr<Buffer> buffer;
+    ARROW_RETURN_NOT_OK(reader->ReadMetadata(&buffer));
+  }
+
+  std::unique_ptr<FlightInfo> info;
+  ARROW_RETURN_NOT_OK(
+      internal::FlightClientImpl_GetFlightInfo(*client_, options_, descriptor, 
&info));
+
+  return std::move(info);
+}
+
+arrow::Result<int64_t> 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());
+  const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command);
+  std::unique_ptr<FlightStreamWriter> writer;
+  std::unique_ptr<FlightMetadataReader> 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(writer->WriteRecordBatch(*parameter_binding_));
+  } else {
+    const std::shared_ptr<Schema> schema = arrow::schema({});
+    ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoPut(*client_, options_, 
descriptor,
+                                                         schema, &writer, 
&reader));
+    const auto& record_batch =
+        arrow::RecordBatch::Make(schema, 0, 
(std::vector<std::shared_ptr<Array>>){});
+    ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*record_batch));
+  }
+
+  ARROW_RETURN_NOT_OK(writer->DoneWriting());
+  std::shared_ptr<Buffer> metadata;
+  ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata));
+  ARROW_RETURN_NOT_OK(writer->Close());
+
+  pb::sql::DoPutUpdateResult result;
+  if (!result.ParseFromArray(metadata->data(), 
static_cast<int>(metadata->size()))) {
+    return Status::Invalid("Unable to parse DoPutUpdateResult object.");
+  }
+
+  return result.record_count();
+}
+
+Status FlightSqlClient::PreparedStatement::SetParameters(
+    std::shared_ptr<RecordBatch> parameter_binding) {
+  parameter_binding_ = std::move(parameter_binding);
+
+  return Status::OK();
+}
+
+bool FlightSqlClient::PreparedStatement::IsClosed() const { return is_closed_; 
}
+
+arrow::Result<std::shared_ptr<Schema>>
+FlightSqlClient::PreparedStatement::GetResultSetSchema() {

Review comment:
       Also if we do the parsing in `Prepare` I think we can get rid of having 
to pass the Protobuf to the constructor here, which obviates the comment about 
not exposing the Protobuf types in the public header.

##########
File path: cpp/src/arrow/flight/flight-sql/server.h
##########
@@ -0,0 +1,493 @@
+// 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 <arrow/flight/server.h>
+#include <sqlite3.h>
+
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+
+#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; };

Review comment:
       why the `using` here? It should all be able to be `struct StatmentQuery 
{...};`.

##########
File path: cpp/src/arrow/flight/flight-sql/server.cc
##########
@@ -0,0 +1,672 @@
+// 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/server.h"
+
+#include <arrow/flight/flight-sql/FlightSql.pb.h>
+#include <google/protobuf/any.pb.h>
+
+#include <boost/algorithm/string.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <sstream>
+
+#include "arrow/api.h"
+#include "arrow/buffer.h"
+
+std::string 
arrow::flight::sql::FlightSqlServerBase::CreateStatementQueryTicket(

Review comment:
       Why not move this into the `namespace` declarations?

##########
File path: cpp/src/arrow/flight/flight-sql/server.h
##########
@@ -0,0 +1,493 @@
+// 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 <arrow/flight/server.h>
+#include <sqlite3.h>
+
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+
+#include "arrow/api.h"

Review comment:
       nit: try not to include "arrow/api.h" in a public header as it can slow 
compilation significantly - either include the headers needed, using 
`type_fwd.h` where possible, or forward-declare things as needed.

##########
File path: cpp/src/arrow/flight/flight-sql/server.h
##########
@@ -0,0 +1,493 @@
+// 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 <arrow/flight/server.h>
+#include <sqlite3.h>

Review comment:
       Did we mean to depend on sqlite3.h here?

##########
File path: cpp/src/arrow/flight/flight-sql/server.h
##########
@@ -0,0 +1,493 @@
+// 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 <arrow/flight/server.h>
+#include <sqlite3.h>
+
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+
+#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 {
+  std::string statement_handle;
+};
+
+using PreparedStatementQuery = struct PreparedStatementQuery {
+  std::string prepared_statement_handle;
+};
+
+using PreparedStatementUpdate = struct PreparedStatementUpdate {
+  std::string prepared_statement_handle;
+};
+
+using GetSqlInfo = struct GetSqlInfo {
+  // TODO: To be implemented.
+};
+
+using GetSchemas = struct GetSchemas {
+  bool has_catalog;
+  std::string catalog;
+  bool has_schema_filter_pattern;
+  std::string schema_filter_pattern;
+};
+
+using GetTables = 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;
+  std::vector<std::string> table_types;
+  bool include_schema;
+};
+
+using GetPrimaryKeys = struct GetPrimaryKeys {
+  bool has_catalog;
+  std::string catalog;
+  bool has_schema;
+  std::string schema;
+  std::string table;
+};
+
+using GetExportedKeys = struct GetExportedKeys {
+  bool has_catalog;
+  std::string catalog;
+  bool has_schema;
+  std::string schema;
+  std::string table;
+};
+
+using GetImportedKeys = struct GetImportedKeys {
+  bool has_catalog;
+  std::string catalog;
+  bool has_schema;
+  std::string schema;
+  std::string table;
+};
+
+using GetCrossReference = struct GetCrossReference {
+  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 {
+  std::string query;
+};
+
+using ActionClosePreparedStatementRequest = struct 
ActionClosePreparedStatementRequest {
+  std::string prepared_statement_handle;
+};
+
+using ActionCreatePreparedStatementResult = struct 
ActionCreatePreparedStatementResult {
+  std::shared_ptr<Schema> dataset_schema;
+  std::shared_ptr<Schema> parameter_schema;
+  std::string prepared_statement_handle;
+};
+
+class FlightSqlServerBase : public FlightServerBase {
+ public:
+  Status GetFlightInfo(const ServerCallContext& context, const 
FlightDescriptor& request,
+                       std::unique_ptr<FlightInfo>* info) override;
+
+  Status DoGet(const ServerCallContext& context, const Ticket& request,
+               std::unique_ptr<FlightDataStream>* stream) override;
+
+  Status DoPut(const ServerCallContext& context,
+               std::unique_ptr<FlightMessageReader> reader,
+               std::unique_ptr<FlightMetadataWriter> writer) override;
+
+  const ActionType FLIGHT_SQL_CREATE_PREPARED_STATEMENT =

Review comment:
       nit: the naming convention for constants in this codebase should be 
`kFlightSqlCreatePreparedStatement`. But since we're in FlightSQL already I 
would probably name this something like `kCreatePreparedStatementActionType`.

##########
File path: cpp/src/arrow/flight/flight-sql/client.h
##########
@@ -0,0 +1,233 @@
+// 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.pb.h>
+#include <arrow/flight/client.h>
+#include <arrow/flight/flight-sql/FlightSql.pb.h>
+#include <arrow/flight/types.h>
+#include <arrow/result.h>
+#include <arrow/status.h>
+#include <google/protobuf/message.h>
+
+namespace pb = arrow::flight::protocol;
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+namespace internal {
+class FlightClientImpl;
+}
+
+/// \brief Flight client with Flight SQL semantics.
+class ARROW_EXPORT FlightSqlClient {
+ private:
+  std::shared_ptr<internal::FlightClientImpl> impl_;
+
+ public:
+  class PreparedStatement;
+
+  explicit FlightSqlClient(std::shared_ptr<internal::FlightClientImpl> client);
+
+  ~FlightSqlClient();
+
+  /// \brief Execute a query on the server.
+  /// \param[in] options      RPC-layer hints for this call.
+  /// \param[in] query        The query to be executed in the UTF-8 format.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \return The quantity of rows affected by the operation.
+  arrow::Result<int64_t> ExecuteUpdate(const FlightCallOptions& options,
+                                       const std::string& query) const;
+
+  /// \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<std::unique_ptr<FlightInfo>> 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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
+  /// \return The returned RecordBatchReader
+  arrow::Result<std::unique_ptr<FlightStreamReader>> DoGet(
+      const FlightCallOptions& options, const Ticket& ticket) const;
+
+  /// \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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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<std::string>& 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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
+  ///        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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> GetSqlInfo(
+      const FlightCallOptions& options, const std::vector<int>& 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> GetSqlInfo(
+      const FlightCallOptions& options,
+      const std::vector<pb::sql::SqlInfo>& 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.
+  /// \return The created prepared statement.
+  arrow::Result<std::shared_ptr<PreparedStatement>> Prepare(
+      const FlightCallOptions& options, const std::string& query);
+
+  explicit FlightSqlClient(std::unique_ptr<FlightClient> client);
+
+  class PreparedStatement {

Review comment:
       Maybe un-nest this class from FlightSqlClient?

##########
File path: cpp/src/arrow/flight/flight-sql/server.h
##########
@@ -0,0 +1,493 @@
+// 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 <arrow/flight/server.h>
+#include <sqlite3.h>
+
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+
+#include "arrow/api.h"

Review comment:
       It seems you should need only `arrow/status.h` from a quick glance.

##########
File path: cpp/src/arrow/flight/flight-sql/server.h
##########
@@ -0,0 +1,493 @@
+// 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 <arrow/flight/server.h>
+#include <sqlite3.h>
+
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+
+#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 {
+  std::string statement_handle;
+};
+
+using PreparedStatementQuery = struct PreparedStatementQuery {
+  std::string prepared_statement_handle;
+};
+
+using PreparedStatementUpdate = struct PreparedStatementUpdate {
+  std::string prepared_statement_handle;
+};
+
+using GetSqlInfo = struct GetSqlInfo {
+  // TODO: To be implemented.
+};
+
+using GetSchemas = struct GetSchemas {
+  bool has_catalog;
+  std::string catalog;

Review comment:
       We can include `arrow/util/optional.h` and use 
`util::optional<std::string>` to clean this up.

##########
File path: cpp/src/arrow/flight/flight-sql/client.h
##########
@@ -0,0 +1,233 @@
+// 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.pb.h>
+#include <arrow/flight/client.h>
+#include <arrow/flight/flight-sql/FlightSql.pb.h>
+#include <arrow/flight/types.h>
+#include <arrow/result.h>
+#include <arrow/status.h>
+#include <google/protobuf/message.h>
+
+namespace pb = arrow::flight::protocol;
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+namespace internal {
+class FlightClientImpl;
+}
+
+/// \brief Flight client with Flight SQL semantics.
+class ARROW_EXPORT FlightSqlClient {
+ private:
+  std::shared_ptr<internal::FlightClientImpl> impl_;
+
+ public:
+  class PreparedStatement;
+
+  explicit FlightSqlClient(std::shared_ptr<internal::FlightClientImpl> client);
+
+  ~FlightSqlClient();
+
+  /// \brief Execute a query on the server.
+  /// \param[in] options      RPC-layer hints for this call.
+  /// \param[in] query        The query to be executed in the UTF-8 format.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \return The quantity of rows affected by the operation.
+  arrow::Result<int64_t> ExecuteUpdate(const FlightCallOptions& options,
+                                       const std::string& query) const;
+
+  /// \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<std::unique_ptr<FlightInfo>> 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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
+  /// \return The returned RecordBatchReader
+  arrow::Result<std::unique_ptr<FlightStreamReader>> DoGet(
+      const FlightCallOptions& options, const Ticket& ticket) const;
+
+  /// \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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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<std::string>& 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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
+  ///        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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> GetSqlInfo(
+      const FlightCallOptions& options, const std::vector<int>& 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> GetSqlInfo(
+      const FlightCallOptions& options,
+      const std::vector<pb::sql::SqlInfo>& 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.
+  /// \return The created prepared statement.
+  arrow::Result<std::shared_ptr<PreparedStatement>> Prepare(
+      const FlightCallOptions& options, const std::string& query);
+
+  explicit FlightSqlClient(std::unique_ptr<FlightClient> client);
+
+  class PreparedStatement {
+    std::shared_ptr<internal::FlightClientImpl> client_;
+    FlightCallOptions options_;
+    pb::sql::ActionCreatePreparedStatementResult prepared_statement_result_;
+    std::shared_ptr<RecordBatch> 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<internal::FlightClientImpl> 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();

Review comment:
       I think we should note that it's best to explicitly close the statement 
or else errors can't be caught.

##########
File path: cpp/src/arrow/flight/flight-sql/server.cc
##########
@@ -0,0 +1,672 @@
+// 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/server.h"
+
+#include <arrow/flight/flight-sql/FlightSql.pb.h>
+#include <google/protobuf/any.pb.h>
+
+#include <boost/algorithm/string.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <sstream>
+
+#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;
+
+arrow::Result<GetCrossReference> ParseCommandGetCrossReference(
+    const google::protobuf::Any& any) {
+  pb::sql::CommandGetCrossReference command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack CommandGetCrossReference.");
+  }
+
+  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;
+}
+
+arrow::Result<GetImportedKeys> ParseCommandGetImportedKeys(
+    const google::protobuf::Any& any) {
+  pb::sql::CommandGetImportedKeys command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack CommandGetImportedKeys.");
+  }
+
+  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;
+}
+
+arrow::Result<GetExportedKeys> ParseCommandGetExportedKeys(
+    const google::protobuf::Any& any) {
+  pb::sql::CommandGetExportedKeys command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack CommandGetExportedKeys.");
+  }
+
+  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;
+}
+
+arrow::Result<GetPrimaryKeys> ParseCommandGetPrimaryKeys(
+    const google::protobuf::Any& any) {
+  pb::sql::CommandGetPrimaryKeys command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack CommandGetPrimaryKeys.");
+  }
+
+  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;
+}
+
+arrow::Result<GetSqlInfo> ParseCommandGetSqlInfo(const google::protobuf::Any& 
any) {
+  pb::sql::CommandGetSqlInfo command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack CommandGetSqlInfo.");
+  }
+
+  GetSqlInfo result;
+  return result;
+}
+
+arrow::Result<GetSchemas> ParseCommandGetSchemas(const google::protobuf::Any& 
any) {
+  pb::sql::CommandGetSchemas command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack CommandGetSchemas.");
+  }
+
+  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;
+}
+
+arrow::Result<PreparedStatementQuery> ParseCommandPreparedStatementQuery(
+    const google::protobuf::Any& any) {
+  pb::sql::CommandPreparedStatementQuery command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack CommandPreparedStatementQuery.");
+  }
+
+  PreparedStatementQuery result;
+  result.prepared_statement_handle = command.prepared_statement_handle();
+  return result;
+}
+
+arrow::Result<StatementQuery> ParseCommandStatementQuery(
+    const google::protobuf::Any& any) {
+  pb::sql::CommandStatementQuery command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack CommandStatementQuery.");
+  }
+
+  StatementQuery result;
+  result.query = command.query();
+  return result;
+}
+
+arrow::Result<GetTables> ParseCommandGetTables(const google::protobuf::Any& 
any) {
+  pb::sql::CommandGetTables command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack CommandGetTables.");
+  }
+
+  std::vector<std::string> table_types;
+  table_types.reserve(command.table_types_size());
+  for (const auto& item : command.table_types()) {
+    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.table_types = table_types;
+  result.include_schema = command.include_schema();
+  return result;
+}
+
+arrow::Result<StatementQueryTicket> ParseStatementQueryTicket(
+    const google::protobuf::Any& any) {
+  pb::sql::TicketStatementQuery command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack TicketStatementQuery.");
+  }
+
+  StatementQueryTicket result;
+  result.statement_handle = command.statement_handle();
+  return result;
+}
+
+arrow::Result<StatementUpdate> ParseCommandStatementUpdate(
+    const google::protobuf::Any& any) {
+  pb::sql::CommandStatementUpdate command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack CommandStatementUpdate.");
+  }
+
+  StatementUpdate result;
+  result.query = command.query();
+  return result;
+}
+
+arrow::Result<PreparedStatementUpdate> ParseCommandPreparedStatementUpdate(
+    const google::protobuf::Any& any) {
+  pb::sql::CommandPreparedStatementUpdate command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack CommandPreparedStatementUpdate.");
+  }
+
+  PreparedStatementUpdate result;
+  result.prepared_statement_handle = command.prepared_statement_handle();
+  return result;
+}
+
+arrow::Result<ActionCreatePreparedStatementRequest>
+ParseActionCreatePreparedStatementRequest(const google::protobuf::Any& any) {
+  pb::sql::ActionCreatePreparedStatementRequest command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack 
ActionCreatePreparedStatementRequest.");
+  }
+
+  ActionCreatePreparedStatementRequest result;
+  result.query = command.query();
+  return result;
+}
+
+arrow::Result<ActionClosePreparedStatementRequest>
+ParseActionClosePreparedStatementRequest(const google::protobuf::Any& any) {
+  pb::sql::ActionClosePreparedStatementRequest command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack 
ActionClosePreparedStatementRequest.");
+  }
+
+  ActionClosePreparedStatementRequest result;
+  result.prepared_statement_handle = command.prepared_statement_handle();
+  return result;
+}
+
+Status FlightSqlServerBase::GetFlightInfo(const ServerCallContext& context,
+                                          const FlightDescriptor& request,
+                                          std::unique_ptr<FlightInfo>* info) {
+  google::protobuf::Any any;
+  if (!any.ParseFromArray(request.cmd.data(), 
static_cast<int>(request.cmd.size()))) {
+    return Status::Invalid("Unable to parse command");
+  }
+
+  if (any.Is<pb::sql::CommandStatementQuery>()) {
+    ARROW_ASSIGN_OR_RAISE(StatementQuery internal_command,
+                          ParseCommandStatementQuery(any));
+    return GetFlightInfoStatement(internal_command, context, request, info);
+  } else if (any.Is<pb::sql::CommandPreparedStatementQuery>()) {
+    ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command,
+                          ParseCommandPreparedStatementQuery(any));
+    return GetFlightInfoPreparedStatement(internal_command, context, request, 
info);
+  } else if (any.Is<pb::sql::CommandGetCatalogs>()) {
+    return GetFlightInfoCatalogs(context, request, info);
+  } else if (any.Is<pb::sql::CommandGetSchemas>()) {
+    ARROW_ASSIGN_OR_RAISE(GetSchemas internal_command, 
ParseCommandGetSchemas(any));
+    return GetFlightInfoSchemas(internal_command, context, request, info);
+  } else if (any.Is<pb::sql::CommandGetTables>()) {
+    ARROW_ASSIGN_OR_RAISE(GetTables command, ParseCommandGetTables(any));
+    return GetFlightInfoTables(command, context, request, info);
+  } else if (any.Is<pb::sql::CommandGetTableTypes>()) {
+    return GetFlightInfoTableTypes(context, request, info);
+  } else if (any.Is<pb::sql::CommandGetSqlInfo>()) {
+    ARROW_ASSIGN_OR_RAISE(GetSqlInfo internal_command, 
ParseCommandGetSqlInfo(any));
+    return GetFlightInfoSqlInfo(internal_command, context, request, info);
+  } else if (any.Is<pb::sql::CommandGetPrimaryKeys>()) {
+    ARROW_ASSIGN_OR_RAISE(GetPrimaryKeys internal_command,
+                          ParseCommandGetPrimaryKeys(any));
+    return GetFlightInfoPrimaryKeys(internal_command, context, request, info);
+  } else if (any.Is<pb::sql::CommandGetExportedKeys>()) {
+    ARROW_ASSIGN_OR_RAISE(GetExportedKeys internal_command,
+                          ParseCommandGetExportedKeys(any));
+    return GetFlightInfoExportedKeys(internal_command, context, request, info);
+  } else if (any.Is<pb::sql::CommandGetImportedKeys>()) {
+    ARROW_ASSIGN_OR_RAISE(GetImportedKeys internal_command,
+                          ParseCommandGetImportedKeys(any));
+    return GetFlightInfoImportedKeys(internal_command, context, request, info);
+  } else if (any.Is<pb::sql::CommandGetCrossReference>()) {
+    ARROW_ASSIGN_OR_RAISE(GetCrossReference internal_command,
+                          ParseCommandGetCrossReference(any));
+    return GetFlightInfoCrossReference(internal_command, context, request, 
info);
+  }
+
+  return Status::Invalid("The defined request is invalid.");
+}
+
+Status FlightSqlServerBase::DoGet(const ServerCallContext& context, const 
Ticket& request,
+                                  std::unique_ptr<FlightDataStream>* stream) {
+  google::protobuf::Any any;
+
+  if (!any.ParseFromArray(request.ticket.data(),
+                          static_cast<int>(request.ticket.size()))) {
+    return Status::Invalid("Unable to parse ticket.");
+  }
+
+  if (any.Is<pb::sql::TicketStatementQuery>()) {
+    ARROW_ASSIGN_OR_RAISE(StatementQueryTicket command, 
ParseStatementQueryTicket(any));
+    return DoGetStatement(command, context, stream);
+  } else if (any.Is<pb::sql::CommandPreparedStatementQuery>()) {
+    ARROW_ASSIGN_OR_RAISE(PreparedStatementQuery internal_command,
+                          ParseCommandPreparedStatementQuery(any));
+    return DoGetPreparedStatement(internal_command, context, stream);
+  } else if (any.Is<pb::sql::CommandGetCatalogs>()) {
+    return DoGetCatalogs(context, stream);
+  } else if (any.Is<pb::sql::CommandGetSchemas>()) {
+    ARROW_ASSIGN_OR_RAISE(GetSchemas internal_command, 
ParseCommandGetSchemas(any));
+    return DoGetSchemas(internal_command, context, stream);
+  } else if (any.Is<pb::sql::CommandGetTables>()) {
+    ARROW_ASSIGN_OR_RAISE(GetTables command, ParseCommandGetTables(any));
+    return DoGetTables(command, context, stream);
+  } else if (any.Is<pb::sql::CommandGetTableTypes>()) {
+    return DoGetTableTypes(context, stream);
+  } else if (any.Is<pb::sql::CommandGetSqlInfo>()) {
+    ARROW_ASSIGN_OR_RAISE(GetSqlInfo internal_command, 
ParseCommandGetSqlInfo(any));
+    return DoGetSqlInfo(internal_command, context, stream);
+  } else if (any.Is<pb::sql::CommandGetPrimaryKeys>()) {
+    ARROW_ASSIGN_OR_RAISE(GetPrimaryKeys internal_command,
+                          ParseCommandGetPrimaryKeys(any));
+    return DoGetPrimaryKeys(internal_command, context, stream);
+  } else if (any.Is<pb::sql::CommandGetExportedKeys>()) {
+    ARROW_ASSIGN_OR_RAISE(GetExportedKeys internal_command,
+                          ParseCommandGetExportedKeys(any));
+    return DoGetExportedKeys(internal_command, context, stream);
+  } else if (any.Is<pb::sql::CommandGetImportedKeys>()) {
+    ARROW_ASSIGN_OR_RAISE(GetImportedKeys internal_command,
+                          ParseCommandGetImportedKeys(any));
+    return DoGetImportedKeys(internal_command, context, stream);
+  } else if (any.Is<pb::sql::CommandGetCrossReference>()) {
+    ARROW_ASSIGN_OR_RAISE(GetCrossReference internal_command,
+                          ParseCommandGetCrossReference(any));
+    return DoGetCrossReference(internal_command, context, stream);
+  }
+
+  return Status::Invalid("The defined request is invalid.");
+}
+
+Status FlightSqlServerBase::DoPut(const ServerCallContext& context,
+                                  std::unique_ptr<FlightMessageReader> reader,
+                                  std::unique_ptr<FlightMetadataWriter> 
writer) {
+  const FlightDescriptor& request = reader->descriptor();
+
+  google::protobuf::Any any;
+  any.ParseFromArray(request.cmd.data(), static_cast<int>(request.cmd.size()));

Review comment:
       Check the return value here?

##########
File path: cpp/src/arrow/flight/flight-sql/server_test.cc
##########
@@ -0,0 +1,653 @@
+// 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/api.h>
+#include <arrow/flight/api.h>
+#include <arrow/flight/flight-sql/api.h>
+#include <arrow/flight/flight-sql/example/sqlite_server.h>
+#include <arrow/flight/flight-sql/server.h>
+#include <arrow/flight/test_util.h>
+#include <arrow/flight/types.h>
+#include <arrow/testing/gtest_util.h>
+#include <gmock/gmock.h>
+#include <google/protobuf/any.pb.h>
+#include <gtest/gtest.h>
+
+#define unparen(...) __VA_ARGS__
+#define DECLARE_ARRAY(ARRAY_NAME, TYPE_CLASS, DATA)     \
+  std::shared_ptr<arrow::TYPE_CLASS##Array> 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<arrow::BinaryArray> 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<arrow::TYPE_CLASS##Array> 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;
+
+namespace pb = arrow::flight::protocol;
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+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));

Review comment:
       ^still wondering about this.

##########
File path: cpp/src/arrow/flight/flight-sql/client.cc
##########
@@ -0,0 +1,442 @@
+// 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/buffer.h>
+#include <arrow/flight/flight-sql/FlightSql.pb.h>
+#include <arrow/flight/flight-sql/client.h>
+#include <arrow/flight/flight-sql/client_internal.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>
+#include <google/protobuf/any.pb.h>
+
+#include <memory>
+#include <utility>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+FlightSqlClient::FlightSqlClient(std::shared_ptr<internal::FlightClientImpl> 
client)
+    : impl_(std::move(client)) {}
+
+FlightSqlClient::FlightSqlClient(std::unique_ptr<FlightClient> client)
+    : impl_(internal::FlightClientImpl_Create(std::move(client))) {}
+
+FlightSqlClient::PreparedStatement::PreparedStatement(
+    std::shared_ptr<internal::FlightClientImpl> 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;
+
+FlightSqlClient::PreparedStatement::~PreparedStatement() {
+  const Status status = Close();
+  if (!status.ok()) {
+    ARROW_LOG(ERROR) << "Failed to delete PreparedStatement: " << 
status.ToString();
+  }
+}
+
+inline FlightDescriptor GetFlightDescriptorForCommand(
+    const google::protobuf::Message& command) {
+  google::protobuf::Any any;
+  any.PackFrom(command);
+
+  const std::string& string = any.SerializeAsString();
+  return FlightDescriptor::Command(string);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> GetFlightInfoForCommand(
+    internal::FlightClientImpl& client, const FlightCallOptions& options,
+    const google::protobuf::Message& command) {
+  const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command);
+
+  std::unique_ptr<FlightInfo> flight_info;
+  ARROW_RETURN_NOT_OK(internal::FlightClientImpl_GetFlightInfo(client, options,
+                                                               descriptor, 
&flight_info));
+
+  return std::move(flight_info);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::Execute(
+    const FlightCallOptions& options, const std::string& query) const {
+  pb::sql::CommandStatementQuery command;
+  command.set_query(query);
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<int64_t> FlightSqlClient::ExecuteUpdate(const FlightCallOptions& 
options,
+                                                      const std::string& 
query) const {
+  pb::sql::CommandStatementUpdate command;
+  command.set_query(query);
+
+  const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command);
+
+  std::unique_ptr<FlightStreamWriter> writer;
+  std::unique_ptr<FlightMetadataReader> reader;
+
+  ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoPut(*impl_, options, 
descriptor,
+                                                       NULLPTR, &writer, 
&reader));
+
+  std::shared_ptr<Buffer> metadata;
+
+  ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata));
+
+  pb::sql::DoPutUpdateResult doPutUpdateResult;
+
+  pb::sql::DoPutUpdateResult result;
+  if (!result.ParseFromArray(metadata->data(), 
static_cast<int>(metadata->size()))) {
+    return Status::Invalid("Unable to parse DoPutUpdateResult object.");
+  }
+
+  return result.record_count();
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::GetCatalogs(
+    const FlightCallOptions& options) const {
+  pb::sql::CommandGetCatalogs command;
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::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);
+  }
+  if (schema_filter_pattern != NULLPTR) {
+    command.set_schema_filter_pattern(*schema_filter_pattern);
+  }
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> 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<std::string>& table_types) const {
+  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(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::GetPrimaryKeys(
+    const FlightCallOptions& options, const std::string* catalog,
+    const std::string* schema, const std::string& table) const {
+  pb::sql::CommandGetPrimaryKeys command;
+
+  if (catalog != NULLPTR) {
+    command.set_catalog(*catalog);
+  }
+
+  if (schema != NULLPTR) {
+    command.set_schema(*schema);
+  }
+
+  command.set_table(table);
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::GetExportedKeys(
+    const FlightCallOptions& options, const std::string* catalog,
+    const std::string* schema, const std::string& table) const {
+  pb::sql::CommandGetExportedKeys command;
+
+  if (catalog != NULLPTR) {
+    command.set_catalog(*catalog);
+  }
+
+  if (schema != NULLPTR) {
+    command.set_schema(*schema);
+  }
+
+  command.set_table(table);
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::GetImportedKeys(
+    const FlightCallOptions& options, const std::string* catalog,
+    const std::string* schema, const std::string& table) const {
+  pb::sql::CommandGetImportedKeys command;
+
+  if (catalog != NULLPTR) {
+    command.set_catalog(*catalog);
+  }
+
+  if (schema != NULLPTR) {
+    command.set_schema(*schema);
+  }
+
+  command.set_table(table);
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> 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 {
+  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(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> FlightSqlClient::GetTableTypes(
+    const FlightCallOptions& options) const {
+  pb::sql::CommandGetTableTypes command;
+
+  return GetFlightInfoForCommand(*impl_, options, command);
+}
+
+arrow::Result<std::unique_ptr<FlightStreamReader>> FlightSqlClient::DoGet(
+    const FlightCallOptions& options, const Ticket& ticket) const {
+  std::unique_ptr<FlightStreamReader> stream;
+  ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoGet(*impl_, options, 
ticket, &stream));
+
+  return std::move(stream);
+}
+
+arrow::Result<std::shared_ptr<FlightSqlClient::PreparedStatement>>
+FlightSqlClient::Prepare(const FlightCallOptions& options, const std::string& 
query) {
+  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<ResultStream> results;
+
+  ARROW_RETURN_NOT_OK(
+      internal::FlightClientImpl_DoAction(*impl_, options, action, &results));
+
+  std::unique_ptr<Result> result;
+  ARROW_RETURN_NOT_OK(results->Next(&result));
+
+  google::protobuf::Any prepared_result;
+
+  std::shared_ptr<Buffer> message = std::move(result->body);
+  if (!prepared_result.ParseFromArray(message->data(),
+                                      static_cast<int>(message->size()))) {
+    return Status::Invalid("Unable to parse packed 
ActionCreatePreparedStatementResult");
+  }
+
+  pb::sql::ActionCreatePreparedStatementResult prepared_statement_result;
+
+  if (!prepared_result.UnpackTo(&prepared_statement_result)) {
+    return Status::Invalid("Unable to unpack 
ActionCreatePreparedStatementResult");
+  }
+
+  return std::make_shared<PreparedStatement>(impl_, query, 
prepared_statement_result,
+                                             options);
+}
+
+arrow::Result<std::unique_ptr<FlightInfo>> 
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());
+
+  google::protobuf::Any any;
+  any.PackFrom(execute_query_command);
+
+  const std::string& string = any.SerializeAsString();
+  const FlightDescriptor descriptor = FlightDescriptor::Command(string);
+
+  if (parameter_binding_ && parameter_binding_->num_rows() > 0) {
+    std::unique_ptr<FlightStreamWriter> writer;
+    std::unique_ptr<FlightMetadataReader> 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->DoneWriting());
+    // Wait for the server to ack the result
+    std::shared_ptr<Buffer> buffer;
+    ARROW_RETURN_NOT_OK(reader->ReadMetadata(&buffer));
+  }
+
+  std::unique_ptr<FlightInfo> info;
+  ARROW_RETURN_NOT_OK(
+      internal::FlightClientImpl_GetFlightInfo(*client_, options_, descriptor, 
&info));
+
+  return std::move(info);
+}
+
+arrow::Result<int64_t> 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());
+  const FlightDescriptor& descriptor = GetFlightDescriptorForCommand(command);
+  std::unique_ptr<FlightStreamWriter> writer;
+  std::unique_ptr<FlightMetadataReader> 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(writer->WriteRecordBatch(*parameter_binding_));
+  } else {
+    const std::shared_ptr<Schema> schema = arrow::schema({});
+    ARROW_RETURN_NOT_OK(internal::FlightClientImpl_DoPut(*client_, options_, 
descriptor,
+                                                         schema, &writer, 
&reader));
+    const auto& record_batch =
+        arrow::RecordBatch::Make(schema, 0, 
(std::vector<std::shared_ptr<Array>>){});
+    ARROW_RETURN_NOT_OK(writer->WriteRecordBatch(*record_batch));
+  }
+
+  ARROW_RETURN_NOT_OK(writer->DoneWriting());
+  std::shared_ptr<Buffer> metadata;
+  ARROW_RETURN_NOT_OK(reader->ReadMetadata(&metadata));
+  ARROW_RETURN_NOT_OK(writer->Close());
+
+  pb::sql::DoPutUpdateResult result;
+  if (!result.ParseFromArray(metadata->data(), 
static_cast<int>(metadata->size()))) {
+    return Status::Invalid("Unable to parse DoPutUpdateResult object.");
+  }
+
+  return result.record_count();
+}
+
+Status FlightSqlClient::PreparedStatement::SetParameters(
+    std::shared_ptr<RecordBatch> parameter_binding) {
+  parameter_binding_ = std::move(parameter_binding);
+
+  return Status::OK();
+}
+
+bool FlightSqlClient::PreparedStatement::IsClosed() const { return is_closed_; 
}
+
+arrow::Result<std::shared_ptr<Schema>>
+FlightSqlClient::PreparedStatement::GetResultSetSchema() {

Review comment:
       IMO, we should parse the schemas up front in the constructor, then make 
these const getters like `std::shared_ptr<Schema> result_set_schema() const;`. 
Since we need to handle errors, we could pImpl `PreparedStatement` and handle 
it in a static factory, or do the parsing in `Prepare`.

##########
File path: cpp/src/arrow/flight/flight-sql/CMakeLists.txt
##########
@@ -0,0 +1,116 @@
+# 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")
+
+set(PROTO_DEPENDS ${FLIGHT_SQL_PROTO} ${ARROW_PROTOBUF_LIBPROTOBUF})
+
+add_custom_command(OUTPUT ${FLIGHT_SQL_GENERATED_PROTO_FILES}
+                   COMMAND ${ARROW_PROTOBUF_PROTOC} 
"-I${FLIGHT_SQL_PROTO_PATH}"
+                           "--cpp_out=${CMAKE_CURRENT_BINARY_DIR}" 
"${FLIGHT_SQL_PROTO}"
+                   DEPENDS ${PROTO_DEPENDS})
+
+set_source_files_properties(${FLIGHT_SQL_GENERATED_PROTO_FILES} PROPERTIES 
GENERATED TRUE)
+
+add_custom_target(flight_sql_protobuf_gen ALL DEPENDS 
${FLIGHT_SQL_GENERATED_PROTO_FILES})
+
+set(ARROW_FLIGHT_SQL_SRCS server.cc client.cc client_internal.cc)
+
+add_arrow_lib(arrow_flight_sql
+              CMAKE_PACKAGE_NAME
+              ArrowFlightSql
+              PKG_CONFIG_NAME
+              arrow-flight-sql
+              OUTPUTS
+              ARROW_FLIGHT_SQL_LIBRARIES
+              SOURCES
+              ${ARROW_FLIGHT_SQL_SRCS}
+              PRECOMPILED_HEADERS
+              "$<$<COMPILE_LANGUAGE:CXX>:arrow/flight/flight-sql/pch.h>"
+              DEPENDENCIES
+              flight_sql_protobuf_gen
+              SHARED_LINK_FLAGS
+              ${ARROW_VERSION_SCRIPT_FLAGS} # Defined in 
cpp/arrow/CMakeLists.txt
+              SHARED_LINK_LIBS
+              arrow_flight_shared
+              ${ARROW_FLIGHT_SQL_LINK_LIBS}

Review comment:
       (Or was it meant to be ARROW_FLIGHT_LINK_LIBS?)

##########
File path: cpp/src/arrow/flight/flight-sql/client_test.cc
##########
@@ -0,0 +1,569 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/flight/client.h>
+#include <arrow/flight/flight-sql/FlightSql.pb.h>
+#include <arrow/flight/flight-sql/api.h>
+#include <arrow/testing/gtest_util.h>
+#include <gmock/gmock.h>
+#include <google/protobuf/any.pb.h>
+#include <gtest/gtest.h>
+
+#include <utility>
+
+namespace pb = arrow::flight::protocol;
+using ::testing::_;
+using ::testing::Ref;
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+namespace internal {
+class FlightClientImpl {
+ public:
+  ~FlightClientImpl() = default;
+
+  MOCK_METHOD(Status, GetFlightInfo,
+              (const FlightCallOptions&, const FlightDescriptor&,
+               std::unique_ptr<FlightInfo>*));
+  MOCK_METHOD(Status, DoGet,
+              (const FlightCallOptions& options, const Ticket& ticket,
+               std::unique_ptr<FlightStreamReader>* stream));
+  MOCK_METHOD(Status, DoPut,
+              (const FlightCallOptions&, const FlightDescriptor&,
+               const std::shared_ptr<Schema>& schema,
+               std::unique_ptr<FlightStreamWriter>*,
+               std::unique_ptr<FlightMetadataReader>*));
+  MOCK_METHOD(Status, DoAction,
+              (const FlightCallOptions& options, const Action& action,
+               std::unique_ptr<ResultStream>* results));
+};
+
+Status FlightClientImpl_GetFlightInfo(FlightClientImpl* client,
+                                      const FlightCallOptions& options,
+                                      const FlightDescriptor& descriptor,
+                                      std::unique_ptr<FlightInfo>* info) {
+  return client->GetFlightInfo(options, descriptor, info);
+}
+
+Status FlightClientImpl_DoPut(FlightClientImpl* client, const 
FlightCallOptions& options,
+                              const FlightDescriptor& descriptor,
+                              const std::shared_ptr<Schema>& schema,
+                              std::unique_ptr<FlightStreamWriter>* stream,
+                              std::unique_ptr<FlightMetadataReader>* reader) {
+  return client->DoPut(options, descriptor, schema, stream, reader);
+}
+
+Status FlightClientImpl_DoGet(FlightClientImpl* client, const 
FlightCallOptions& options,
+                              const Ticket& ticket,
+                              std::unique_ptr<FlightStreamReader>* stream) {
+  return client->DoGet(options, ticket, stream);
+}
+
+Status FlightClientImpl_DoAction(FlightClientImpl* client,
+                                 const FlightCallOptions& options, const 
Action& action,
+                                 std::unique_ptr<ResultStream>* results) {
+  return client->DoAction(options, action, results);
+}
+
+}  // namespace internal
+
+class FlightMetadataReaderMock : public FlightMetadataReader {
+ public:
+  std::shared_ptr<Buffer>* buffer;
+
+  explicit FlightMetadataReaderMock(std::shared_ptr<Buffer>* buffer) {
+    this->buffer = buffer;
+  }
+
+  Status ReadMetadata(std::shared_ptr<Buffer>* out) override {
+    *out = *buffer;
+    return Status::OK();
+  }
+};
+
+class FlightStreamWriterMock : public FlightStreamWriter {
+ public:
+  FlightStreamWriterMock() = default;
+
+  Status DoneWriting() override { return Status::OK(); }
+
+  Status WriteMetadata(std::shared_ptr<Buffer> app_metadata) override {
+    return Status::OK();
+  }
+
+  Status Begin(const std::shared_ptr<Schema>& schema,
+               const ipc::IpcWriteOptions& options) override {
+    return Status::OK();
+  }
+
+  Status Begin(const std::shared_ptr<Schema>& schema) override {
+    return MetadataRecordBatchWriter::Begin(schema);
+  }
+
+  ipc::WriteStats stats() const override { return ipc::WriteStats(); }
+
+  Status WriteWithMetadata(const RecordBatch& batch,
+                           std::shared_ptr<Buffer> app_metadata) override {
+    return Status::OK();
+  }
+
+  Status Close() override { return Status::OK(); }
+
+  Status WriteRecordBatch(const RecordBatch& batch) override { return 
Status::OK(); }
+};
+
+FlightDescriptor getDescriptor(google::protobuf::Message& command) {
+  google::protobuf::Any any;
+  any.PackFrom(command);
+
+  const std::string& string = any.SerializeAsString();
+  return FlightDescriptor::Command(string);
+}
+
+TEST(TestFlightSqlClient, TestGetCatalogs) {
+  auto client_mock = std::make_shared<internal::FlightClientImpl>();
+  FlightSqlClient sql_client(client_mock);

Review comment:
       Uh, hmm. Is this legal? It feels like we're violating ODR…though it 
certainly appears to work. 

##########
File path: cpp/src/arrow/flight/flight-sql/server.h
##########
@@ -0,0 +1,493 @@
+// 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 <arrow/flight/server.h>
+#include <sqlite3.h>
+
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+
+#include "arrow/api.h"

Review comment:
       (Though, please add `<string>`, `<memory>`, etc. for std::string, 
std::shared_ptr, std::unique_ptr, et. al.)

##########
File path: cpp/src/arrow/flight/flight-sql/server.h
##########
@@ -0,0 +1,493 @@
+// 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 <arrow/flight/server.h>
+#include <sqlite3.h>
+
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+
+#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 {
+  std::string statement_handle;
+};
+
+using PreparedStatementQuery = struct PreparedStatementQuery {
+  std::string prepared_statement_handle;
+};
+
+using PreparedStatementUpdate = struct PreparedStatementUpdate {
+  std::string prepared_statement_handle;
+};
+
+using GetSqlInfo = struct GetSqlInfo {
+  // TODO: To be implemented.
+};
+
+using GetSchemas = struct GetSchemas {
+  bool has_catalog;
+  std::string catalog;
+  bool has_schema_filter_pattern;
+  std::string schema_filter_pattern;
+};
+
+using GetTables = 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;
+  std::vector<std::string> table_types;
+  bool include_schema;
+};
+
+using GetPrimaryKeys = struct GetPrimaryKeys {
+  bool has_catalog;
+  std::string catalog;
+  bool has_schema;
+  std::string schema;
+  std::string table;
+};
+
+using GetExportedKeys = struct GetExportedKeys {
+  bool has_catalog;
+  std::string catalog;
+  bool has_schema;
+  std::string schema;
+  std::string table;
+};
+
+using GetImportedKeys = struct GetImportedKeys {
+  bool has_catalog;
+  std::string catalog;
+  bool has_schema;
+  std::string schema;
+  std::string table;
+};
+
+using GetCrossReference = struct GetCrossReference {
+  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 {
+  std::string query;
+};
+
+using ActionClosePreparedStatementRequest = struct 
ActionClosePreparedStatementRequest {
+  std::string prepared_statement_handle;
+};
+
+using ActionCreatePreparedStatementResult = struct 
ActionCreatePreparedStatementResult {
+  std::shared_ptr<Schema> dataset_schema;
+  std::shared_ptr<Schema> parameter_schema;
+  std::string prepared_statement_handle;
+};
+
+class FlightSqlServerBase : public FlightServerBase {

Review comment:
       I think this is needed for Windows?

##########
File path: cpp/src/arrow/flight/flight-sql/client.h
##########
@@ -0,0 +1,233 @@
+// 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.pb.h>
+#include <arrow/flight/client.h>
+#include <arrow/flight/flight-sql/FlightSql.pb.h>
+#include <arrow/flight/types.h>
+#include <arrow/result.h>
+#include <arrow/status.h>
+#include <google/protobuf/message.h>
+
+namespace pb = arrow::flight::protocol;
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+namespace internal {
+class FlightClientImpl;
+}
+
+/// \brief Flight client with Flight SQL semantics.
+class ARROW_EXPORT FlightSqlClient {
+ private:
+  std::shared_ptr<internal::FlightClientImpl> impl_;
+
+ public:
+  class PreparedStatement;
+
+  explicit FlightSqlClient(std::shared_ptr<internal::FlightClientImpl> client);
+
+  ~FlightSqlClient();
+
+  /// \brief Execute a query on the server.
+  /// \param[in] options      RPC-layer hints for this call.
+  /// \param[in] query        The query to be executed in the UTF-8 format.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \return The quantity of rows affected by the operation.
+  arrow::Result<int64_t> ExecuteUpdate(const FlightCallOptions& options,
+                                       const std::string& query) const;
+
+  /// \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<std::unique_ptr<FlightInfo>> 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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
+  /// \return The returned RecordBatchReader
+  arrow::Result<std::unique_ptr<FlightStreamReader>> DoGet(
+      const FlightCallOptions& options, const Ticket& ticket) const;
+
+  /// \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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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<std::string>& 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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
+  ///        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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> GetSqlInfo(
+      const FlightCallOptions& options, const std::vector<int>& 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.
+  /// \return The FlightInfo describing where to access the dataset.
+  arrow::Result<std::unique_ptr<FlightInfo>> GetSqlInfo(
+      const FlightCallOptions& options,
+      const std::vector<pb::sql::SqlInfo>& 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.
+  /// \return The created prepared statement.
+  arrow::Result<std::shared_ptr<PreparedStatement>> Prepare(
+      const FlightCallOptions& options, const std::string& query);
+
+  explicit FlightSqlClient(std::unique_ptr<FlightClient> client);
+
+  class PreparedStatement {
+    std::shared_ptr<internal::FlightClientImpl> client_;
+    FlightCallOptions options_;
+    pb::sql::ActionCreatePreparedStatementResult prepared_statement_result_;
+    std::shared_ptr<RecordBatch> 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<internal::FlightClientImpl> client_, const 
std::string& query,
+        pb::sql::ActionCreatePreparedStatementResult& 
prepared_statement_result,

Review comment:
       This is still leaking the Protobuf types into the public API - perhaps 
this class needs to be pImpl'd?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

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


Reply via email to