From 18adc86aeb7563f6914d3ed531599895429078f2 Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky Date: Thu, 12 Oct 2023 17:54:30 -0400 Subject: [PATCH 01/13] Update flight sql proto Clarify schema, catalog behavior in comment Remove target_ prefix Add comment explaining extensions to spec Fix type of transaction_id Generate go pb Add to server Add to client Unimplemented ingest test Implement record writing Add integration test Add sqlinfo type Fix proto comment indent Client drain read stream, ensure single result Split out integration test Cleanup scenario file Skip cpp runner Move ingest tx sqlinfo Regen go pb Remove unused method Close recordwriter Add c++ impl Make schema nullable in go integration test Add ingest tx sqlinfo Update sqlinfo name in proto Regenerate go pb Update sqlinfo name in go Run linter Fix stylistic issues Extract IngestMode to types.h Use std::optional for optional parameters Run linter Add sqlinfo value for bulk ingestion Fix linting issues Raise explicit client error Add docs Clarify how DoPutUpdateResult may be accessed Update sqlinfo numbers Update comment for 'temporary' Elaborate scenarios for temporary Update format/FlightSql.proto Co-authored-by: Antoine Pitrou Regenerate proto with updated comment Update semantics for "replace" temp unspecified equivalent to false Clarify transactionid Temporary is not optional Replace ingestmode with table_definition_options Run lint Linting exceptions for generated pb lines Regenerate go pb --- .../flight_integration_test.cc | 4 + .../integration_tests/test_integration.cc | 125 ++ cpp/src/arrow/flight/sql/client.cc | 108 ++ cpp/src/arrow/flight/sql/client.h | 18 + cpp/src/arrow/flight/sql/server.cc | 103 ++ cpp/src/arrow/flight/sql/server.h | 27 + cpp/src/arrow/flight/sql/types.h | 31 + dev/archery/archery/integration/runner.py | 5 + docs/source/format/FlightSql.rst | 16 + .../FlightSql/CommandStatementIngest.mmd | 28 + .../FlightSql/CommandStatementIngest.mmd.svg | 1 + format/FlightSql.proto | 82 +- go/arrow/flight/flightsql/client.go | 52 + go/arrow/flight/flightsql/server.go | 30 + go/arrow/flight/flightsql/server_test.go | 36 + go/arrow/flight/flightsql/types.go | 42 + go/arrow/flight/gen/flight/FlightSql.pb.go | 1302 ++++++++++------- go/arrow/flight/gen/flight/Flight_grpc.pb.go | 56 +- .../internal/flight_integration/scenario.go | 136 ++ 19 files changed, 1638 insertions(+), 564 deletions(-) create mode 100644 docs/source/format/FlightSql/CommandStatementIngest.mmd create mode 100644 docs/source/format/FlightSql/CommandStatementIngest.mmd.svg diff --git a/cpp/src/arrow/flight/integration_tests/flight_integration_test.cc b/cpp/src/arrow/flight/integration_tests/flight_integration_test.cc index 92c088b7fae..15318a8d7a4 100644 --- a/cpp/src/arrow/flight/integration_tests/flight_integration_test.cc +++ b/cpp/src/arrow/flight/integration_tests/flight_integration_test.cc @@ -89,6 +89,10 @@ TEST(FlightIntegration, FlightSqlExtension) { ASSERT_OK(RunScenario("flight_sql:extension")); } +TEST(FlightIntegration, FlightSqlIngestion) { + ASSERT_OK(RunScenario("flight_sql:ingestion")); +} + } // namespace integration_tests } // namespace flight } // namespace arrow diff --git a/cpp/src/arrow/flight/integration_tests/test_integration.cc b/cpp/src/arrow/flight/integration_tests/test_integration.cc index 21f57efd122..d0a7124de13 100644 --- a/cpp/src/arrow/flight/integration_tests/test_integration.cc +++ b/cpp/src/arrow/flight/integration_tests/test_integration.cc @@ -1068,6 +1068,7 @@ constexpr int64_t kUpdateStatementExpectedRows = 10000L; constexpr int64_t kUpdateStatementWithTransactionExpectedRows = 15000L; constexpr int64_t kUpdatePreparedStatementExpectedRows = 20000L; constexpr int64_t kUpdatePreparedStatementWithTransactionExpectedRows = 25000L; +constexpr int64_t kIngestStatementExpectedRows = 3L; constexpr char kSelectStatement[] = "SELECT STATEMENT"; constexpr char kSavepointId[] = "savepoint_id"; constexpr char kSavepointName[] = "savepoint_name"; @@ -2123,6 +2124,127 @@ class ReuseConnectionScenario : public Scenario { return Status::OK(); } }; + +std::shared_ptr GetIngestSchema() { + return arrow::schema({arrow::field("test_field", arrow::int64(), true)}); +} + +arrow::Result> GetIngestRecords() { + auto schema = GetIngestSchema(); + auto array = arrow::ArrayFromJSON(arrow::int64(), "[null,null,null]"); + auto record_batch = arrow::RecordBatch::Make(schema, 3, {array}); + return RecordBatchReader::Make({record_batch}); +} + +/// \brief The server used for testing bulk ingestion +class FlightSqlIngestionServer : public sql::FlightSqlServerBase { + public: + FlightSqlIngestionServer() : sql::FlightSqlServerBase() { + RegisterSqlInfo(sql::SqlInfoOptions::SqlInfo::FLIGHT_SQL_SERVER_BULK_INGESTION, + sql::SqlInfoResult(true)); + RegisterSqlInfo( + sql::SqlInfoOptions::SqlInfo::FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED, + sql::SqlInfoResult(true)); + } + + arrow::Result DoPutCommandStatementIngest( + const ServerCallContext& context, const sql::StatementIngest& command, + FlightMessageReader* reader) override { + ARROW_RETURN_NOT_OK(AssertEq( + true, + sql::TableDefinitionOptionsTableNotExistOption::kCreate == + command.table_definition_options.if_not_exist, + "Wrong TableDefinitionOptionsTableNotExistOption for ExecuteIngest")); + ARROW_RETURN_NOT_OK(AssertEq( + true, + sql::TableDefinitionOptionsTableExistsOption::kReplace == + command.table_definition_options.if_exists, + "Wrong TableDefinitionOptionsTableExistsOption for ExecuteIngest")); + ARROW_RETURN_NOT_OK(AssertEq("test_table", command.table, + "Wrong table for ExecuteIngest")); + ARROW_RETURN_NOT_OK(AssertEq("test_schema", command.schema.value(), + "Wrong schema for ExecuteIngest")); + ARROW_RETURN_NOT_OK(AssertEq("test_catalog", command.catalog.value(), + "Wrong catalog for ExecuteIngest")); + ARROW_RETURN_NOT_OK(AssertEq(true, command.temporary, + "Wrong temporary setting for ExecuteIngest")); + ARROW_RETURN_NOT_OK(AssertEq("123", command.transaction_id.value(), + "Wrong transaction_id for ExecuteIngest")); + + std::unordered_map expected_options = {{"key1", "val1"}, + {"key2", "val2"}}; + ARROW_RETURN_NOT_OK( + AssertEq(expected_options.size(), command.options.size(), + "Wrong number of options set for ExecuteIngest")); + for (auto it = expected_options.begin(); it != expected_options.end(); ++it) { + auto key = it->first; + auto expected_val = it->second; + ARROW_RETURN_NOT_OK( + AssertEq(expected_val, command.options.at(key), + "Wrong option value set for ExecuteIngest")); + } + + auto expected_schema = GetIngestSchema(); + int64_t num_records = 0; + while (true) { + ARROW_ASSIGN_OR_RAISE(FlightStreamChunk chunk, reader->Next()); + if (chunk.data == nullptr) break; + + ARROW_RETURN_NOT_OK( + AssertEq(true, expected_schema->Equals(chunk.data->schema()), + "Chunk schema does not match expected schema for ExecuteIngest")); + num_records += chunk.data->num_rows(); + } + + return num_records; + } +}; + +/// \brief The FlightSqlIngestion scenario. +/// +/// This tests that the client can execute bulk ingestion against the server. +/// +/// The server implements DoPutCommandStatementIngest and validates that the arguments +/// it receives are the same as those supplied to the client, or have been successfully +/// mapped to the equivalent server-side representation. The size and schema of the sent +/// and received streams are also validated against eachother. +class FlightSqlIngestionScenario : public Scenario { + Status MakeServer(std::unique_ptr* server, + FlightServerOptions* options) override { + server->reset(new FlightSqlIngestionServer()); + return Status::OK(); + } + + Status MakeClient(FlightClientOptions* options) override { return Status::OK(); } + + Status RunClient(std::unique_ptr client) override { + sql::FlightSqlClient sql_client(std::move(client)); + ARROW_RETURN_NOT_OK(ValidateIngestion(&sql_client)); + return Status::OK(); + } + + Status ValidateIngestion(sql::FlightSqlClient* sql_client) { + ARROW_ASSIGN_OR_RAISE(auto record_batch_reader, GetIngestRecords()); + + sql::TableDefinitionOptions table_definition_options; + table_definition_options.if_not_exist = + sql::TableDefinitionOptionsTableNotExistOption::kCreate; + table_definition_options.if_exists = + sql::TableDefinitionOptionsTableExistsOption::kReplace; + bool temporary = true; + std::unordered_map options = {{"key1", "val1"}, + {"key2", "val2"}}; + ARROW_ASSIGN_OR_RAISE( + auto updated_rows, + sql_client->ExecuteIngest({}, record_batch_reader, table_definition_options, + "test_table", "test_schema", "test_catalog", temporary, + sql::Transaction("123"), options)); + ARROW_RETURN_NOT_OK(AssertEq(kIngestStatementExpectedRows, updated_rows, + "Wrong number of updated rows for ExecuteIngest")); + + return Status::OK(); + } +}; } // namespace Status GetScenario(const std::string& scenario_name, std::shared_ptr* out) { @@ -2165,6 +2287,9 @@ Status GetScenario(const std::string& scenario_name, std::shared_ptr* } else if (scenario_name == "flight_sql:extension") { *out = std::make_shared(); return Status::OK(); + } else if (scenario_name == "flight_sql:ingestion") { + *out = std::make_shared(); + return Status::OK(); } return Status::KeyError("Scenario not found: ", scenario_name); } diff --git a/cpp/src/arrow/flight/sql/client.cc b/cpp/src/arrow/flight/sql/client.cc index d0552e33df5..37b6a0b32e4 100644 --- a/cpp/src/arrow/flight/sql/client.cc +++ b/cpp/src/arrow/flight/sql/client.cc @@ -256,6 +256,114 @@ arrow::Result FlightSqlClient::ExecuteSubstraitUpdate( return update_result.record_count(); } +arrow::Result FlightSqlClient::ExecuteIngest( + const FlightCallOptions& options, const std::shared_ptr& reader, + const TableDefinitionOptions& table_definition_options, const std::string& table, + const std::optional& schema, const std::optional& catalog, + const bool temporary, const Transaction& transaction, + const std::unordered_map& ingest_options) { + flight_sql_pb::CommandStatementIngest command; + + flight_sql_pb::CommandStatementIngest_TableDefinitionOptions* + pb_table_definition_options = + new flight_sql_pb::CommandStatementIngest_TableDefinitionOptions(); + switch (table_definition_options.if_not_exist) { + case TableDefinitionOptionsTableNotExistOption::kUnspecified: + pb_table_definition_options->set_if_not_exist( + flight_sql_pb:: + CommandStatementIngest_TableDefinitionOptions_TableNotExistOption_TABLE_NOT_EXIST_OPTION_UNSPECIFIED); // NOLINT(whitespace/line_length) + break; + case TableDefinitionOptionsTableNotExistOption::kCreate: + pb_table_definition_options->set_if_not_exist( + flight_sql_pb:: + CommandStatementIngest_TableDefinitionOptions_TableNotExistOption_TABLE_NOT_EXIST_OPTION_CREATE); // NOLINT(whitespace/line_length) + break; + case TableDefinitionOptionsTableNotExistOption::kFail: + pb_table_definition_options->set_if_not_exist( + flight_sql_pb:: + CommandStatementIngest_TableDefinitionOptions_TableNotExistOption_TABLE_NOT_EXIST_OPTION_FAIL); // NOLINT(whitespace/line_length) + break; + + default: + break; + } + + switch (table_definition_options.if_exists) { + case TableDefinitionOptionsTableExistsOption::kUnspecified: + pb_table_definition_options->set_if_exists( + flight_sql_pb:: + CommandStatementIngest_TableDefinitionOptions_TableExistsOption_TABLE_EXISTS_OPTION_UNSPECIFIED); // NOLINT(whitespace/line_length) + break; + case TableDefinitionOptionsTableExistsOption::kFail: + pb_table_definition_options->set_if_exists( + flight_sql_pb:: + CommandStatementIngest_TableDefinitionOptions_TableExistsOption_TABLE_EXISTS_OPTION_FAIL); // NOLINT(whitespace/line_length) + break; + case TableDefinitionOptionsTableExistsOption::kAppend: + pb_table_definition_options->set_if_exists( + flight_sql_pb:: + CommandStatementIngest_TableDefinitionOptions_TableExistsOption_TABLE_EXISTS_OPTION_APPEND); // NOLINT(whitespace/line_length) + break; + case TableDefinitionOptionsTableExistsOption::kReplace: + pb_table_definition_options->set_if_exists( + flight_sql_pb:: + CommandStatementIngest_TableDefinitionOptions_TableExistsOption_TABLE_EXISTS_OPTION_REPLACE); // NOLINT(whitespace/line_length) + break; + + default: + break; + } + + command.set_allocated_table_definition_options(pb_table_definition_options); + command.set_table(table); + + if (schema.has_value()) { + command.set_schema(schema.value()); + } + + if (catalog.has_value()) { + command.set_catalog(catalog.value()); + } + + command.set_temporary(temporary); + + if (transaction.is_valid()) { + command.set_transaction_id(transaction.transaction_id()); + } + + auto command_options = command.mutable_options(); + for (const auto& [key, val] : ingest_options) { + (*command_options)[key] = val; + } + + ARROW_ASSIGN_OR_RAISE(FlightDescriptor descriptor, + GetFlightDescriptorForCommand(command)); + + auto reader_ = reader.get(); + ARROW_ASSIGN_OR_RAISE(auto stream, DoPut(options, descriptor, reader_->schema())); + + while (true) { + ARROW_ASSIGN_OR_RAISE(auto batch, reader_->Next()); + if (!batch) break; + ARROW_RETURN_NOT_OK(stream.writer->WriteRecordBatch(*batch)); + } + + ARROW_RETURN_NOT_OK(stream.writer->DoneWriting()); + std::shared_ptr metadata; + ARROW_RETURN_NOT_OK(stream.reader->ReadMetadata(&metadata)); + ARROW_RETURN_NOT_OK(stream.writer->Close()); + + if (!metadata) return Status::IOError("Server did not send a response"); + + flight_sql_pb::DoPutUpdateResult update_result; + if (!update_result.ParseFromArray(metadata->data(), + static_cast(metadata->size()))) { + return Status::Invalid("Unable to parse DoPutUpdateResult"); + } + + return update_result.record_count(); +} + arrow::Result> FlightSqlClient::GetCatalogs( const FlightCallOptions& options) { flight_sql_pb::CommandGetCatalogs command; diff --git a/cpp/src/arrow/flight/sql/client.h b/cpp/src/arrow/flight/sql/client.h index 9782611dbad..c37c640e653 100644 --- a/cpp/src/arrow/flight/sql/client.h +++ b/cpp/src/arrow/flight/sql/client.h @@ -101,6 +101,24 @@ class ARROW_FLIGHT_SQL_EXPORT FlightSqlClient { const FlightCallOptions& options, const SubstraitPlan& plan, const Transaction& transaction = no_transaction()); + /// \brief Execute a bulk ingestion to the server. + /// \param[in] options RPC-layer hints for this call. + /// \param[in] reader The records to ingest. + /// \param[in] table_definition_options The behavior for handling the table definition. + /// \param[in] table The destination table to load into. + /// \param[in] schema The DB schema of the destination table. + /// \param[in] catalog The catalog of the destination table. + /// \param[in] temporary Use a temporary table. + /// \param[in] transaction Ingest as part of this transaction. + /// \param[in] ingest_options Additional, backend-specific options. + /// \return The number of rows ingested to the server. + arrow::Result ExecuteIngest( + const FlightCallOptions& options, const std::shared_ptr& reader, + const TableDefinitionOptions& table_definition_options, const std::string& table, + const std::optional& schema, const std::optional& catalog, + const bool temporary, const Transaction& transaction = no_transaction(), + const std::unordered_map& ingest_options = {}); + /// \brief Request a list of catalogs. /// \param[in] options RPC-layer hints for this call. /// \return The FlightInfo describing where to access the dataset. diff --git a/cpp/src/arrow/flight/sql/server.cc b/cpp/src/arrow/flight/sql/server.cc index a5cb842de8f..cae3542b4fa 100644 --- a/cpp/src/arrow/flight/sql/server.cc +++ b/cpp/src/arrow/flight/sql/server.cc @@ -234,6 +234,90 @@ arrow::Result ParseCommandPreparedStatementUpdate( return result; } +arrow::Result ParseCommandStatementIngest( + const google::protobuf::Any& any) { + pb::sql::CommandStatementIngest command; + if (!any.UnpackTo(&command)) { + return Status::Invalid("Unable to unpack CommandStatementIngest."); + } + + StatementIngest result; + TableDefinitionOptions table_definition_options; + switch (command.table_definition_options().if_not_exist()) { + case pb::sql::CommandStatementIngest::TableDefinitionOptions:: + TABLE_NOT_EXIST_OPTION_UNSPECIFIED: + table_definition_options.if_not_exist = + TableDefinitionOptionsTableNotExistOption::kUnspecified; + break; + case pb::sql::CommandStatementIngest::TableDefinitionOptions:: + TABLE_NOT_EXIST_OPTION_CREATE: + table_definition_options.if_not_exist = + TableDefinitionOptionsTableNotExistOption::kCreate; + break; + case pb::sql::CommandStatementIngest::TableDefinitionOptions:: + TABLE_NOT_EXIST_OPTION_FAIL: + table_definition_options.if_not_exist = + TableDefinitionOptionsTableNotExistOption::kFail; + break; + + default: + return Status::Invalid( + "Unrecognized TableNotExistOption for " + "CommandStatementIngest::TableDefinitionOptions."); + } + + switch (command.table_definition_options().if_exists()) { + case pb::sql::CommandStatementIngest::TableDefinitionOptions:: + TABLE_EXISTS_OPTION_UNSPECIFIED: + table_definition_options.if_exists = + TableDefinitionOptionsTableExistsOption::kUnspecified; + break; + case pb::sql::CommandStatementIngest::TableDefinitionOptions:: + TABLE_EXISTS_OPTION_FAIL: + table_definition_options.if_exists = TableDefinitionOptionsTableExistsOption::kFail; + break; + case pb::sql::CommandStatementIngest::TableDefinitionOptions:: + TABLE_EXISTS_OPTION_APPEND: + table_definition_options.if_exists = + TableDefinitionOptionsTableExistsOption::kAppend; + break; + case pb::sql::CommandStatementIngest::TableDefinitionOptions:: + TABLE_EXISTS_OPTION_REPLACE: + table_definition_options.if_exists = + TableDefinitionOptionsTableExistsOption::kReplace; + break; + + default: + return Status::Invalid( + "Unrecognized TableExistsOption for " + "CommandStatementIngest::TableDefinitionOptions."); + } + + result.table_definition_options = table_definition_options; + result.table = command.table(); + + if (command.has_schema()) { + result.schema = command.schema(); + } + + if (command.has_catalog()) { + result.catalog = command.catalog(); + } + + result.temporary = command.temporary(); + + if (command.has_transaction_id()) { + result.transaction_id = command.transaction_id(); + } + + std::unordered_map options; + for (const auto& [key, val] : command.options()) { + options[key] = val; + } + result.options = options; + return result; +} + arrow::Result ParseActionBeginSavepointRequest( const google::protobuf::Any& any) { pb::sql::ActionBeginSavepointRequest command; @@ -764,6 +848,19 @@ Status FlightSqlServerBase::DoPut(const ServerCallContext& context, pb::sql::DoPutUpdateResult result; result.set_record_count(record_count); + const auto buffer = Buffer::FromString(result.SerializeAsString()); + ARROW_RETURN_NOT_OK(writer->WriteMetadata(*buffer)); + return Status::OK(); + } else if (any.Is()) { + ARROW_ASSIGN_OR_RAISE(StatementIngest internal_command, + ParseCommandStatementIngest(any)); + ARROW_ASSIGN_OR_RAISE( + auto record_count, + DoPutCommandStatementIngest(context, internal_command, reader.get())); + + pb::sql::DoPutUpdateResult result; + result.set_record_count(record_count); + const auto buffer = Buffer::FromString(result.SerializeAsString()); ARROW_RETURN_NOT_OK(writer->WriteMetadata(*buffer)); return Status::OK(); @@ -1202,6 +1299,12 @@ arrow::Result FlightSqlServerBase::DoPutCommandSubstraitPlan( return Status::NotImplemented("DoPutCommandSubstraitPlan not implemented"); } +arrow::Result FlightSqlServerBase::DoPutCommandStatementIngest( + const ServerCallContext& context, const StatementIngest& command, + FlightMessageReader* reader) { + return Status::NotImplemented("DoPutCommandStatementIngest not implemented"); +} + const std::shared_ptr& SqlSchema::GetCatalogsSchema() { static std::shared_ptr kSchema = arrow::schema({field("catalog_name", utf8(), false)}); diff --git a/cpp/src/arrow/flight/sql/server.h b/cpp/src/arrow/flight/sql/server.h index df46004474a..7b5d71678f3 100644 --- a/cpp/src/arrow/flight/sql/server.h +++ b/cpp/src/arrow/flight/sql/server.h @@ -86,6 +86,24 @@ struct ARROW_FLIGHT_SQL_EXPORT PreparedStatementUpdate { std::string prepared_statement_handle; }; +/// \brief A bulk ingestion request +struct ARROW_FLIGHT_SQL_EXPORT StatementIngest { + /// \brief The behavior for handling the table definition. + TableDefinitionOptions table_definition_options; + /// \brief The destination table to load into. + std::string table; + /// \brief The DB schema of the destination table. + std::optional schema; + /// :\brief The catalog of the destination table. + std::optional catalog; + /// \brief Use a temporary table. + bool temporary; + /// \brief Ingest as part of this transaction. + std::optional transaction_id; + /// \brief Additional, backend-specific options. + std::unordered_map options; +}; + /// \brief A request to fetch server metadata. struct ARROW_FLIGHT_SQL_EXPORT GetSqlInfo { /// \brief A list of metadata IDs to fetch. @@ -569,6 +587,15 @@ class ARROW_FLIGHT_SQL_EXPORT FlightSqlServerBase : public FlightServerBase { const ServerCallContext& context, const PreparedStatementUpdate& command, FlightMessageReader* reader); + /// \brief Execute a bulk ingestion. + /// \param[in] context The call context. + /// \param[in] command The StatementIngest object containing the ingestion request. + /// \param[in] reader a sequence of uploaded record batches. + /// \return The changed record count. + virtual arrow::Result DoPutCommandStatementIngest( + const ServerCallContext& context, const StatementIngest& command, + FlightMessageReader* reader); + /// \brief Begin a new transaction. /// \param[in] context The call context. /// \param[in] request Request parameters. diff --git a/cpp/src/arrow/flight/sql/types.h b/cpp/src/arrow/flight/sql/types.h index b41488b68f2..fe90c08ed20 100644 --- a/cpp/src/arrow/flight/sql/types.h +++ b/cpp/src/arrow/flight/sql/types.h @@ -107,6 +107,14 @@ struct ARROW_FLIGHT_SQL_EXPORT SqlInfoOptions { /// supports explicit query cancellation (the CancelQuery action). FLIGHT_SQL_SERVER_CANCEL = 9, + /// Retrieves a boolean value indicating whether the Flight SQL Server + /// supports executing bulk ingestion. + FLIGHT_SQL_SERVER_BULK_INGESTION = 10, + + /// Retrieves a boolean value indicating whether the Flight SQL Server + /// supports transactions for bulk ingestion. + FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED = 11, + /// Retrieves an int32 value indicating the timeout (in milliseconds) for /// prepared statement handles. /// @@ -920,6 +928,29 @@ enum class CancelResult : int8_t { kNotCancellable, }; +/// \brief The action to take if the target table of an ingestion does not exist. +enum class TableDefinitionOptionsTableNotExistOption { + kUnspecified, + kCreate, + kFail, +}; + +/// \brief The action to take if the target table of an ingestion already exists. +enum class TableDefinitionOptionsTableExistsOption { + kUnspecified, + kFail, + kAppend, + kReplace, +}; + +/// \brief Options for table definition behavior of bulk ingestion. +struct TableDefinitionOptions { + /// \brief Behavior when the table does not exist. + TableDefinitionOptionsTableNotExistOption if_not_exist; + /// \brief Behavior when the table exists. + TableDefinitionOptionsTableExistsOption if_exists; +}; + ARROW_FLIGHT_SQL_EXPORT std::ostream& operator<<(std::ostream& os, CancelResult result); diff --git a/dev/archery/archery/integration/runner.py b/dev/archery/archery/integration/runner.py index b50d5c5c264..5b66842b259 100644 --- a/dev/archery/archery/integration/runner.py +++ b/dev/archery/archery/integration/runner.py @@ -638,6 +638,11 @@ def run_all_tests(with_cpp=True, with_java=True, with_js=True, description="Ensure Flight SQL extensions work as expected.", skip_testers={"Rust"} ), + Scenario( + "flight_sql:ingestion", + description="Ensure Flight SQL ingestion works as expected.", + skip_testers={"JS", "C#", "Rust", "Java"} + ), ] runner = IntegrationRunner(json_files, flight_scenarios, testers, **kwargs) diff --git a/docs/source/format/FlightSql.rst b/docs/source/format/FlightSql.rst index 6bb91727136..0f99d77cb93 100644 --- a/docs/source/format/FlightSql.rst +++ b/docs/source/format/FlightSql.rst @@ -114,6 +114,12 @@ google.protobuf.Any message, then serialized and packed into the to the command name (i.e. for ``ActionClosePreparedStatementRequest``, the ``type`` should be ``ClosePreparedStatement``). +Commands that execute updates such as ``CommandStatementUpdate`` and +``CommandStatementIngest`` return a Flight SQL ``DoPutUpdateResult`` +after consuming the entire FlightData stream. This message is encoded +in the ``app_metadata`` field of the Flight RPC ``PutResult`` returned. + + ``ActionClosePreparedStatementRequest`` Close a previously created prepared statement. @@ -170,6 +176,12 @@ the ``type`` should be ``ClosePreparedStatement``). When used with DoPut: execute the query and return the number of affected rows. +``CommandStatementIngest`` + Execute a bulk ingestion. + + When used with DoPut: load the stream of Arrow record batches into + the specified target table and return the number of rows ingested. + Flight Server Session Management -------------------------------- @@ -226,6 +238,10 @@ Sequence Diagrams Creating a prepared statement, then executing it. +.. figure:: ./FlightSql/CommandStatementIngest.mmd.svg + + Executing a bulk ingestion. + External Resources ================== diff --git a/docs/source/format/FlightSql/CommandStatementIngest.mmd b/docs/source/format/FlightSql/CommandStatementIngest.mmd new file mode 100644 index 00000000000..781289d77b4 --- /dev/null +++ b/docs/source/format/FlightSql/CommandStatementIngest.mmd @@ -0,0 +1,28 @@ +%% Licensed to the Apache Software Foundation (ASF) under one +%% or more contributor license agreements. See the NOTICE file +%% distributed with this work for additional information +%% regarding copyright ownership. The ASF licenses this file +%% to you under the Apache License, Version 2.0 (the +%% "License"); you may not use this file except in compliance +%% with the License. You may obtain a copy of the License at +%% +%% http://www.apache.org/licenses/LICENSE-2.0 +%% +%% Unless required by applicable law or agreed to in writing, +%% software distributed under the License is distributed on an +%% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +%% KIND, either express or implied. See the License for the +%% specific language governing permissions and limitations +%% under the License. + +%% To generate the diagram, use mermaid-cli +%% Example: docker run --rm -v $(pwd)/FlightSql:/data minlag/mermaid-cli -i /data/CommandGetTables.mmd + +sequenceDiagram +autonumber + +participant Client +participant Server +Client->>Server: DoPut(CommandStatementIngest) +Client->>Server: stream of FlightData +Server->>Client: PutResult{DoPutUpdateResult{RecordCount: int64}} diff --git a/docs/source/format/FlightSql/CommandStatementIngest.mmd.svg b/docs/source/format/FlightSql/CommandStatementIngest.mmd.svg new file mode 100644 index 00000000000..e2aa72459af --- /dev/null +++ b/docs/source/format/FlightSql/CommandStatementIngest.mmd.svg @@ -0,0 +1 @@ +ServerClientServerClientDoPut(CommandStatementIngest)1stream of FlightData2PutResult{DoPutUpdateResult{RecordCount: int64}}3 \ No newline at end of file diff --git a/format/FlightSql.proto b/format/FlightSql.proto index 581cf1f76d5..39f0f259464 100644 --- a/format/FlightSql.proto +++ b/format/FlightSql.proto @@ -135,6 +135,23 @@ enum SqlInfo { */ FLIGHT_SQL_SERVER_CANCEL = 9; + /* + * Retrieves a boolean value indicating whether the Flight SQL Server supports executing + * bulk ingestion. + */ + FLIGHT_SQL_SERVER_BULK_INGESTION = 10; + + /* + * Retrieves a boolean value indicating whether transactions are supported for bulk ingestion. If not, invoking + * the method commit in the context of a bulk ingestion is a noop, and the isolation level is + * `arrow.flight.protocol.sql.SqlTransactionIsolationLevel.TRANSACTION_NONE`. + * + * Returns: + * - false: if bulk ingestion transactions are unsupported; + * - true: if bulk ingestion transactions are supported. + */ + FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED = 11; + /* * Retrieves an int32 indicating the timeout (in milliseconds) for prepared statement handles. * @@ -1785,9 +1802,68 @@ message CommandPreparedStatementUpdate { } /* - * Returned from the RPC call DoPut when a CommandStatementUpdate - * CommandPreparedStatementUpdate was in the request, containing - * results from the update. + * Represents a bulk ingestion request. Used in the command member of FlightDescriptor + * for the the RPC call DoPut to cause the server load the contents of the stream's + * FlightData into the target destination. + */ +message CommandStatementIngest { + option (experimental) = true; + + // Options for table definition behavior + message TableDefinitionOptions { + // The action to take if the target table does not exist + enum TableNotExistOption { + // Do not use. Servers should error if this is specified by a client. + TABLE_NOT_EXIST_OPTION_UNSPECIFIED = 0; + // Create the table if it does not exist + TABLE_NOT_EXIST_OPTION_CREATE = 1; + // Fail if the table does not exist + TABLE_NOT_EXIST_OPTION_FAIL = 2; + } + // The action to take if the target table already exists + enum TableExistsOption { + // Do not use. Servers should error if this is specified by a client. + TABLE_EXISTS_OPTION_UNSPECIFIED = 0; + // Fail if the table already exists + TABLE_EXISTS_OPTION_FAIL = 1; + // Append to the table if it already exists + TABLE_EXISTS_OPTION_APPEND = 2; + // Drop and recreate the table if it already exists + TABLE_EXISTS_OPTION_REPLACE = 3; + } + + TableNotExistOption if_not_exist = 1; + TableExistsOption if_exists = 2; + } + + // The behavior for handling the table definition. + TableDefinitionOptions table_definition_options = 1; + // The table to load data into. + string table = 2; + // The db_schema of the destination table to load data into. If unset, a backend-specific default may be used. + optional string schema = 3; + // The catalog of the destination table to load data into. If unset, a backend-specific default may be used. + optional string catalog = 4; + /* + * Store ingested data in a temporary table. + * The effect of setting temporary is to place the table in a backend-defined namespace, and to drop the table at the end of the session. + * The namespacing may make use of a backend-specific schema and/or catalog. + * The server should return an error if an explicit choice of schema or catalog is incompatible with the server's namespacing decision. + */ + bool temporary = 5; + // Perform the ingestion as part of this transaction. If specified, results should not be committed in the event of an error/cancellation. + optional bytes transaction_id = 6; + + // Future extensions to the parameters of CommandStatementIngest should be added here, at a lower index than the generic 'options' parameter. + + // Backend-specific options. + map options = 1000; +} + +/* + * Returned from the RPC call DoPut when a CommandStatementUpdate, + * CommandPreparedStatementUpdate, or CommandStatementIngest was + * in the request, containing results from the update. */ message DoPutUpdateResult { option (experimental) = true; diff --git a/go/arrow/flight/flightsql/client.go b/go/arrow/flight/flightsql/client.go index 09e4974ae9b..0ea43765cdd 100644 --- a/go/arrow/flight/flightsql/client.go +++ b/go/arrow/flight/flightsql/client.go @@ -243,6 +243,58 @@ func (c *Client) ExecuteSubstraitUpdate(ctx context.Context, plan SubstraitPlan, return updateResult.GetRecordCount(), nil } +var ErrTooManyPutResults = fmt.Errorf("%w: server sent multiple PutResults, expected one ", arrow.ErrInvalid) + +// ExecuteIngest is for executing a bulk ingestion and only returns the number of affected rows. +func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqOptions *ExecuteIngestOpts, opts ...grpc.CallOption) (n int64, err error) { + var ( + desc *flight.FlightDescriptor + stream pb.FlightService_DoPutClient + wr *flight.Writer + res *pb.PutResult + updateResult pb.DoPutUpdateResult + ) + + cmd := (*pb.CommandStatementIngest)(reqOptions) + if desc, err = descForCommand(cmd); err != nil { + return + } + + if stream, err = c.Client.DoPut(ctx, opts...); err != nil { + return + } + + wr = flight.NewRecordWriter(stream, ipc.WithAllocator(c.Alloc), ipc.WithSchema(rdr.Schema())) + defer wr.Close() + + wr.SetFlightDescriptor(desc) + + for rdr.Next() { + rec := rdr.Record() + wr.Write(rec) + } + + if err = stream.CloseSend(); err != nil { + return + } + + if res, err = stream.Recv(); err != nil { + return + } + + // Drain the stream, ensuring only a single result was sent by the server. + // Flight RPC allows multiple server results, so the check may be removed if Flight SQL ingestion defines semanatics for this scenario. + if _, err = stream.Recv(); err != io.EOF { + return 0, ErrTooManyPutResults + } + + if err = proto.Unmarshal(res.GetAppMetadata(), &updateResult); err != nil { + return + } + + return updateResult.GetRecordCount(), nil +} + // GetCatalogs requests the list of catalogs from the server and // returns a flightInfo object where the response can be retrieved func (c *Client) GetCatalogs(ctx context.Context, opts ...grpc.CallOption) (*flight.FlightInfo, error) { diff --git a/go/arrow/flight/flightsql/server.go b/go/arrow/flight/flightsql/server.go index 7bc15ab4295..a72b79bccdc 100644 --- a/go/arrow/flight/flightsql/server.go +++ b/go/arrow/flight/flightsql/server.go @@ -196,6 +196,17 @@ type ActionEndSavepointRequest interface { GetAction() EndSavepointRequestType } +// StatementIngest represents a bulk ingestion request +type StatementIngest interface { + GetTableDefinitionOptions() *TableDefinitionOptions + GetTable() string + GetSchema() string + GetCatalog() string + GetTemporary() bool + GetTransactionId() []byte + GetOptions() map[string]string +} + type getXdbcTypeInfo struct { *pb.CommandGetXdbcTypeInfo } @@ -507,6 +518,10 @@ func (BaseServer) DoPutPreparedStatementUpdate(context.Context, PreparedStatemen return 0, status.Error(codes.Unimplemented, "DoPutPreparedStatementUpdate not implemented") } +func (BaseServer) DoPutCommandStatementIngest(context.Context, StatementIngest, flight.MessageReader) (int64, error) { + return 0, status.Error(codes.Unimplemented, "DoPutCommandStatementIngest not implemented") +} + func (BaseServer) BeginTransaction(context.Context, ActionBeginTransactionRequest) ([]byte, error) { return nil, status.Error(codes.Unimplemented, "BeginTransaction not implemented") } @@ -694,6 +709,9 @@ type Server interface { GetSessionOptions(context.Context, *flight.GetSessionOptionsRequest) (*flight.GetSessionOptionsResult, error) // CloseSession closes/invalidates the current server session. CloseSession(context.Context, *flight.CloseSessionRequest) (*flight.CloseSessionResult, error) + // DoPutCommandStatementIngest executes a bulk ingestion and returns + // the number of affected rows + DoPutCommandStatementIngest(context.Context, StatementIngest, flight.MessageReader) (int64, error) mustEmbedBaseServer() } @@ -979,6 +997,18 @@ func (f *flightSqlServer) DoPut(stream flight.FlightService_DoPutServer) error { return err } + result := pb.DoPutUpdateResult{RecordCount: recordCount} + out := &flight.PutResult{} + if out.AppMetadata, err = proto.Marshal(&result); err != nil { + return status.Errorf(codes.Internal, "failed to marshal PutResult: %s", err.Error()) + } + return stream.Send(out) + case *pb.CommandStatementIngest: + recordCount, err := f.srv.DoPutCommandStatementIngest(stream.Context(), cmd, rdr) + if err != nil { + return err + } + result := pb.DoPutUpdateResult{RecordCount: recordCount} out := &flight.PutResult{} if out.AppMetadata, err = proto.Marshal(&result); err != nil { diff --git a/go/arrow/flight/flightsql/server_test.go b/go/arrow/flight/flightsql/server_test.go index df619e7a241..125be8ff8c5 100644 --- a/go/arrow/flight/flightsql/server_test.go +++ b/go/arrow/flight/flightsql/server_test.go @@ -28,6 +28,7 @@ import ( "github.com/apache/arrow/go/v16/arrow/flight/flightsql" pb "github.com/apache/arrow/go/v16/arrow/flight/gen/flight" "github.com/apache/arrow/go/v16/arrow/flight/session" + "github.com/apache/arrow/go/v16/arrow/internal/arrdata" "github.com/apache/arrow/go/v16/arrow/memory" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -178,6 +179,15 @@ func (*testServer) CloseSession(ctx context.Context, req *flight.CloseSessionReq return &flight.CloseSessionResult{Status: flight.CloseSessionResultClosed}, nil } +func (*testServer) DoPutCommandStatementIngest(ctx context.Context, cmd flightsql.StatementIngest, rdr flight.MessageReader) (int64, error) { + var nRecords int64 + for rdr.Next() { + rec := rdr.Record() + nRecords += rec.NumRows() + } + return nRecords, nil +} + type FlightSqlServerSuite struct { suite.Suite @@ -281,6 +291,21 @@ func (s *FlightSqlServerSuite) TestExecutePoll() { s.Len(poll.GetInfo().Endpoint, 2) } +func (s *FlightSqlServerSuite) TestExecuteIngest() { + var nRecordsExpected int64 + + reclist := arrdata.Records["primitives"] + for _, rec := range reclist { + nRecordsExpected += rec.NumRows() + } + + rdr, _ := array.NewRecordReader(reclist[0].Schema(), reclist) + nRecords, err := s.cl.ExecuteIngest(context.TODO(), rdr, &flightsql.ExecuteIngestOpts{}) + + s.Require().NoError(err) + s.Equal(nRecordsExpected, nRecords) +} + type UnimplementedFlightSqlServerSuite struct { suite.Suite @@ -459,6 +484,17 @@ func (s *UnimplementedFlightSqlServerSuite) TestDoGet() { } } +func (s *UnimplementedFlightSqlServerSuite) TestExecuteIngest() { + reclist := arrdata.Records["primitives"] + rdr, _ := array.NewRecordReader(reclist[0].Schema(), reclist) + info, err := s.cl.ExecuteIngest(context.TODO(), rdr, &flightsql.ExecuteIngestOpts{}) + st, ok := status.FromError(err) + s.True(ok) + s.Equal(codes.Unimplemented, st.Code()) + s.Equal(st.Message(), "DoPutCommandStatementIngest not implemented") + s.Zero(info) +} + func (s *UnimplementedFlightSqlServerSuite) TestDoAction() { prep, err := s.cl.Prepare(context.TODO(), "IRRELEVANT") s.Nil(prep) diff --git a/go/arrow/flight/flightsql/types.go b/go/arrow/flight/flightsql/types.go index c70a8bdc4ec..265c430670c 100644 --- a/go/arrow/flight/flightsql/types.go +++ b/go/arrow/flight/flightsql/types.go @@ -134,6 +134,20 @@ type ( // the substrait release, e.g. "0.23.0" Version string } + + // ExecuteIngestOpts contains the options for executing a bulk ingestion: + // + // Required: + // - TableDefinitionOptions: Specifies the behavior for creating or updating table definitions + // - Table: The destination table to load into + // + // Optional: + // - Schema: The DB schema containing the destination table + // - Catalog: The catalog containing the destination table + // - Temporary: Use a temporary table as the destination + // - TransactionId: Ingest as part of this transaction + // - Options: Additional, backend-specific options + ExecuteIngestOpts pb.CommandStatementIngest ) // SqlInfo enum values @@ -198,6 +212,18 @@ const ( // If 0, there is no timeout. Servers should reset the timeout when the handle is used in a command. SqlInfoFlightSqlServerTransactionTimeout = SqlInfo(pb.SqlInfo_FLIGHT_SQL_SERVER_TRANSACTION_TIMEOUT) + // Retrieves a boolean value indicating whether the Flight SQL Server supports executing + // bulk ingestion. + SqlInfoFlightSqlServerBulkIngestion = SqlInfo(pb.SqlInfo_FLIGHT_SQL_SERVER_BULK_INGESTION) + // Retrieves a boolean value indicating whether transactions are supported for bulk ingestion. If not, invoking + // the method commit in the context of a bulk ingestion is a noop, and the isolation level is + // `arrow.flight.protocol.sql.SqlTransactionIsolationLevel.TRANSACTION_NONE`. + // + // Returns: + // - false: if bulk ingestion transactions are unsupported; + // - true: if bulk ingestion transactions are supported. + SqlInfoFlightSqlServerIngestTransactionsSupported = SqlInfo(pb.SqlInfo_FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED) + // SQL Syntax Information // Values [500-1000): provide information about the supported SQL Syntax @@ -854,3 +880,19 @@ const ( ) type CreatePreparedStatementResult = pb.ActionCreatePreparedStatementResult + +type TableDefinitionOptions = pb.CommandStatementIngest_TableDefinitionOptions + +type TableDefinitionOptionsTableNotExistOption = pb.CommandStatementIngest_TableDefinitionOptions_TableNotExistOption +type TableDefinitionOptionsTableExistsOption = pb.CommandStatementIngest_TableDefinitionOptions_TableExistsOption + +const ( + TableDefinitionOptionsTableNotExistOptionUnspecified = pb.CommandStatementIngest_TableDefinitionOptions_TABLE_NOT_EXIST_OPTION_UNSPECIFIED + TableDefinitionOptionsTableNotExistOptionCreate = pb.CommandStatementIngest_TableDefinitionOptions_TABLE_NOT_EXIST_OPTION_CREATE + TableDefinitionOptionsTableNotExistOptionFail = pb.CommandStatementIngest_TableDefinitionOptions_TABLE_NOT_EXIST_OPTION_FAIL + + TableDefinitionOptionsTableExistsOptionUnspecified = pb.CommandStatementIngest_TableDefinitionOptions_TABLE_EXISTS_OPTION_UNSPECIFIED + TableDefinitionOptionsTableExistsOptionFail = pb.CommandStatementIngest_TableDefinitionOptions_TABLE_EXISTS_OPTION_FAIL + TableDefinitionOptionsTableExistsOptionAppend = pb.CommandStatementIngest_TableDefinitionOptions_TABLE_EXISTS_OPTION_APPEND + TableDefinitionOptionsTableExistsOptionReplace = pb.CommandStatementIngest_TableDefinitionOptions_TABLE_EXISTS_OPTION_REPLACE +) diff --git a/go/arrow/flight/gen/flight/FlightSql.pb.go b/go/arrow/flight/gen/flight/FlightSql.pb.go index d886bc6bdb7..2a230591a2e 100644 --- a/go/arrow/flight/gen/flight/FlightSql.pb.go +++ b/go/arrow/flight/gen/flight/FlightSql.pb.go @@ -592,6 +592,8 @@ var ( 7: "FLIGHT_SQL_SERVER_SUBSTRAIT_MAX_VERSION", 8: "FLIGHT_SQL_SERVER_TRANSACTION", 9: "FLIGHT_SQL_SERVER_CANCEL", + 10: "FLIGHT_SQL_SERVER_BULK_INGESTION", + 11: "FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED", 100: "FLIGHT_SQL_SERVER_STATEMENT_TIMEOUT", 101: "FLIGHT_SQL_SERVER_TRANSACTION_TIMEOUT", 500: "SQL_DDL_CATALOG", @@ -683,6 +685,8 @@ var ( "FLIGHT_SQL_SERVER_SUBSTRAIT_MAX_VERSION": 7, "FLIGHT_SQL_SERVER_TRANSACTION": 8, "FLIGHT_SQL_SERVER_CANCEL": 9, + "FLIGHT_SQL_SERVER_BULK_INGESTION": 10, + "FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED": 11, "FLIGHT_SQL_SERVER_STATEMENT_TIMEOUT": 100, "FLIGHT_SQL_SERVER_TRANSACTION_TIMEOUT": 101, "SQL_DDL_CATALOG": 500, @@ -2172,6 +2176,116 @@ func (ActionEndSavepointRequest_EndSavepoint) EnumDescriptor() ([]byte, []int) { return file_FlightSql_proto_rawDescGZIP(), []int{20, 0} } +// The action to take if the target table does not exist +type CommandStatementIngest_TableDefinitionOptions_TableNotExistOption int32 + +const ( + // Do not use. Servers should error if this is specified by a client. + CommandStatementIngest_TableDefinitionOptions_TABLE_NOT_EXIST_OPTION_UNSPECIFIED CommandStatementIngest_TableDefinitionOptions_TableNotExistOption = 0 + // Create the table if it does not exist + CommandStatementIngest_TableDefinitionOptions_TABLE_NOT_EXIST_OPTION_CREATE CommandStatementIngest_TableDefinitionOptions_TableNotExistOption = 1 + // Fail if the table does not exist + CommandStatementIngest_TableDefinitionOptions_TABLE_NOT_EXIST_OPTION_FAIL CommandStatementIngest_TableDefinitionOptions_TableNotExistOption = 2 +) + +// Enum value maps for CommandStatementIngest_TableDefinitionOptions_TableNotExistOption. +var ( + CommandStatementIngest_TableDefinitionOptions_TableNotExistOption_name = map[int32]string{ + 0: "TABLE_NOT_EXIST_OPTION_UNSPECIFIED", + 1: "TABLE_NOT_EXIST_OPTION_CREATE", + 2: "TABLE_NOT_EXIST_OPTION_FAIL", + } + CommandStatementIngest_TableDefinitionOptions_TableNotExistOption_value = map[string]int32{ + "TABLE_NOT_EXIST_OPTION_UNSPECIFIED": 0, + "TABLE_NOT_EXIST_OPTION_CREATE": 1, + "TABLE_NOT_EXIST_OPTION_FAIL": 2, + } +) + +func (x CommandStatementIngest_TableDefinitionOptions_TableNotExistOption) Enum() *CommandStatementIngest_TableDefinitionOptions_TableNotExistOption { + p := new(CommandStatementIngest_TableDefinitionOptions_TableNotExistOption) + *p = x + return p +} + +func (x CommandStatementIngest_TableDefinitionOptions_TableNotExistOption) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (CommandStatementIngest_TableDefinitionOptions_TableNotExistOption) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[24].Descriptor() +} + +func (CommandStatementIngest_TableDefinitionOptions_TableNotExistOption) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[24] +} + +func (x CommandStatementIngest_TableDefinitionOptions_TableNotExistOption) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use CommandStatementIngest_TableDefinitionOptions_TableNotExistOption.Descriptor instead. +func (CommandStatementIngest_TableDefinitionOptions_TableNotExistOption) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{27, 0, 0} +} + +// The action to take if the target table already exists +type CommandStatementIngest_TableDefinitionOptions_TableExistsOption int32 + +const ( + // Do not use. Servers should error if this is specified by a client. + CommandStatementIngest_TableDefinitionOptions_TABLE_EXISTS_OPTION_UNSPECIFIED CommandStatementIngest_TableDefinitionOptions_TableExistsOption = 0 + // Fail if the table already exists + CommandStatementIngest_TableDefinitionOptions_TABLE_EXISTS_OPTION_FAIL CommandStatementIngest_TableDefinitionOptions_TableExistsOption = 1 + // Append to the table if it already exists + CommandStatementIngest_TableDefinitionOptions_TABLE_EXISTS_OPTION_APPEND CommandStatementIngest_TableDefinitionOptions_TableExistsOption = 2 + // Drop and recreate the table if it already exists + CommandStatementIngest_TableDefinitionOptions_TABLE_EXISTS_OPTION_REPLACE CommandStatementIngest_TableDefinitionOptions_TableExistsOption = 3 +) + +// Enum value maps for CommandStatementIngest_TableDefinitionOptions_TableExistsOption. +var ( + CommandStatementIngest_TableDefinitionOptions_TableExistsOption_name = map[int32]string{ + 0: "TABLE_EXISTS_OPTION_UNSPECIFIED", + 1: "TABLE_EXISTS_OPTION_FAIL", + 2: "TABLE_EXISTS_OPTION_APPEND", + 3: "TABLE_EXISTS_OPTION_REPLACE", + } + CommandStatementIngest_TableDefinitionOptions_TableExistsOption_value = map[string]int32{ + "TABLE_EXISTS_OPTION_UNSPECIFIED": 0, + "TABLE_EXISTS_OPTION_FAIL": 1, + "TABLE_EXISTS_OPTION_APPEND": 2, + "TABLE_EXISTS_OPTION_REPLACE": 3, + } +) + +func (x CommandStatementIngest_TableDefinitionOptions_TableExistsOption) Enum() *CommandStatementIngest_TableDefinitionOptions_TableExistsOption { + p := new(CommandStatementIngest_TableDefinitionOptions_TableExistsOption) + *p = x + return p +} + +func (x CommandStatementIngest_TableDefinitionOptions_TableExistsOption) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (CommandStatementIngest_TableDefinitionOptions_TableExistsOption) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[25].Descriptor() +} + +func (CommandStatementIngest_TableDefinitionOptions_TableExistsOption) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[25] +} + +func (x CommandStatementIngest_TableDefinitionOptions_TableExistsOption) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use CommandStatementIngest_TableDefinitionOptions_TableExistsOption.Descriptor instead. +func (CommandStatementIngest_TableDefinitionOptions_TableExistsOption) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{27, 0, 1} +} + type ActionCancelQueryResult_CancelResult int32 const ( @@ -2217,11 +2331,11 @@ func (x ActionCancelQueryResult_CancelResult) String() string { } func (ActionCancelQueryResult_CancelResult) Descriptor() protoreflect.EnumDescriptor { - return file_FlightSql_proto_enumTypes[24].Descriptor() + return file_FlightSql_proto_enumTypes[26].Descriptor() } func (ActionCancelQueryResult_CancelResult) Type() protoreflect.EnumType { - return &file_FlightSql_proto_enumTypes[24] + return &file_FlightSql_proto_enumTypes[26] } func (x ActionCancelQueryResult_CancelResult) Number() protoreflect.EnumNumber { @@ -2230,7 +2344,7 @@ func (x ActionCancelQueryResult_CancelResult) Number() protoreflect.EnumNumber { // Deprecated: Use ActionCancelQueryResult_CancelResult.Descriptor instead. func (ActionCancelQueryResult_CancelResult) EnumDescriptor() ([]byte, []int) { - return file_FlightSql_proto_rawDescGZIP(), []int{29, 0} + return file_FlightSql_proto_rawDescGZIP(), []int{30, 0} } // Represents a metadata request. Used in the command member of FlightDescriptor @@ -4200,7 +4314,7 @@ type DoPutUpdateResult struct { func (x *DoPutUpdateResult) Reset() { *x = DoPutUpdateResult{} if protoimpl.UnsafeEnabled { - mi := &file_FlightSql_proto_msgTypes[27] + mi := &file_FlightSql_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4213,7 +4327,7 @@ func (x *DoPutUpdateResult) String() string { func (*DoPutUpdateResult) ProtoMessage() {} func (x *DoPutUpdateResult) ProtoReflect() protoreflect.Message { - mi := &file_FlightSql_proto_msgTypes[27] + mi := &file_FlightSql_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4226,7 +4340,7 @@ func (x *DoPutUpdateResult) ProtoReflect() protoreflect.Message { // Deprecated: Use DoPutUpdateResult.ProtoReflect.Descriptor instead. func (*DoPutUpdateResult) Descriptor() ([]byte, []int) { - return file_FlightSql_proto_rawDescGZIP(), []int{27} + return file_FlightSql_proto_rawDescGZIP(), []int{28} } func (x *DoPutUpdateResult) GetRecordCount() int64 { @@ -4268,7 +4382,7 @@ type ActionCancelQueryRequest struct { func (x *ActionCancelQueryRequest) Reset() { *x = ActionCancelQueryRequest{} if protoimpl.UnsafeEnabled { - mi := &file_FlightSql_proto_msgTypes[28] + mi := &file_FlightSql_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4281,7 +4395,7 @@ func (x *ActionCancelQueryRequest) String() string { func (*ActionCancelQueryRequest) ProtoMessage() {} func (x *ActionCancelQueryRequest) ProtoReflect() protoreflect.Message { - mi := &file_FlightSql_proto_msgTypes[28] + mi := &file_FlightSql_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4294,7 +4408,7 @@ func (x *ActionCancelQueryRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionCancelQueryRequest.ProtoReflect.Descriptor instead. func (*ActionCancelQueryRequest) Descriptor() ([]byte, []int) { - return file_FlightSql_proto_rawDescGZIP(), []int{28} + return file_FlightSql_proto_rawDescGZIP(), []int{29} } func (x *ActionCancelQueryRequest) GetInfo() []byte { @@ -4323,7 +4437,7 @@ type ActionCancelQueryResult struct { func (x *ActionCancelQueryResult) Reset() { *x = ActionCancelQueryResult{} if protoimpl.UnsafeEnabled { - mi := &file_FlightSql_proto_msgTypes[29] + mi := &file_FlightSql_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4336,7 +4450,7 @@ func (x *ActionCancelQueryResult) String() string { func (*ActionCancelQueryResult) ProtoMessage() {} func (x *ActionCancelQueryResult) ProtoReflect() protoreflect.Message { - mi := &file_FlightSql_proto_msgTypes[29] + mi := &file_FlightSql_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4349,7 +4463,7 @@ func (x *ActionCancelQueryResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionCancelQueryResult.ProtoReflect.Descriptor instead. func (*ActionCancelQueryResult) Descriptor() ([]byte, []int) { - return file_FlightSql_proto_rawDescGZIP(), []int{29} + return file_FlightSql_proto_rawDescGZIP(), []int{30} } func (x *ActionCancelQueryResult) GetResult() ActionCancelQueryResult_CancelResult { @@ -4359,6 +4473,62 @@ func (x *ActionCancelQueryResult) GetResult() ActionCancelQueryResult_CancelResu return ActionCancelQueryResult_CANCEL_RESULT_UNSPECIFIED } +// Options for table definition behavior +type CommandStatementIngest_TableDefinitionOptions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + IfNotExist CommandStatementIngest_TableDefinitionOptions_TableNotExistOption `protobuf:"varint,1,opt,name=if_not_exist,json=ifNotExist,proto3,enum=arrow.flight.protocol.sql.CommandStatementIngest_TableDefinitionOptions_TableNotExistOption" json:"if_not_exist,omitempty"` + IfExists CommandStatementIngest_TableDefinitionOptions_TableExistsOption `protobuf:"varint,2,opt,name=if_exists,json=ifExists,proto3,enum=arrow.flight.protocol.sql.CommandStatementIngest_TableDefinitionOptions_TableExistsOption" json:"if_exists,omitempty"` +} + +func (x *CommandStatementIngest_TableDefinitionOptions) Reset() { + *x = CommandStatementIngest_TableDefinitionOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandStatementIngest_TableDefinitionOptions) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandStatementIngest_TableDefinitionOptions) ProtoMessage() {} + +func (x *CommandStatementIngest_TableDefinitionOptions) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[31] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CommandStatementIngest_TableDefinitionOptions.ProtoReflect.Descriptor instead. +func (*CommandStatementIngest_TableDefinitionOptions) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{27, 0} +} + +func (x *CommandStatementIngest_TableDefinitionOptions) GetIfNotExist() CommandStatementIngest_TableDefinitionOptions_TableNotExistOption { + if x != nil { + return x.IfNotExist + } + return CommandStatementIngest_TableDefinitionOptions_TABLE_NOT_EXIST_OPTION_UNSPECIFIED +} + +func (x *CommandStatementIngest_TableDefinitionOptions) GetIfExists() CommandStatementIngest_TableDefinitionOptions_TableExistsOption { + if x != nil { + return x.IfExists + } + return CommandStatementIngest_TableDefinitionOptions_TABLE_EXISTS_OPTION_UNSPECIFIED +} + var file_FlightSql_proto_extTypes = []protoimpl.ExtensionInfo{ { ExtendedType: (*descriptorpb.MessageOptions)(nil), @@ -4638,481 +4808,483 @@ var file_FlightSql_proto_rawDesc = []byte{ 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x41, 0x52, 0x52, 0x4f, 0x57, 0x5f, 0x56, 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x02, 0x12, 0x1f, 0x0a, 0x1b, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, - 0x45, 0x52, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x5f, 0x4f, 0x4e, 0x4c, 0x59, 0x10, 0x03, 0x12, 0x19, - 0x0a, 0x15, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, - 0x56, 0x45, 0x52, 0x5f, 0x53, 0x51, 0x4c, 0x10, 0x04, 0x12, 0x1f, 0x0a, 0x1b, 0x46, 0x4c, 0x49, - 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x53, - 0x55, 0x42, 0x53, 0x54, 0x52, 0x41, 0x49, 0x54, 0x10, 0x05, 0x12, 0x2b, 0x0a, 0x27, 0x46, 0x4c, - 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, - 0x53, 0x55, 0x42, 0x53, 0x54, 0x52, 0x41, 0x49, 0x54, 0x5f, 0x4d, 0x49, 0x4e, 0x5f, 0x56, 0x45, - 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x06, 0x12, 0x2b, 0x0a, 0x27, 0x46, 0x4c, 0x49, 0x47, 0x48, - 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x53, 0x55, 0x42, - 0x53, 0x54, 0x52, 0x41, 0x49, 0x54, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x56, 0x45, 0x52, 0x53, 0x49, - 0x4f, 0x4e, 0x10, 0x07, 0x12, 0x21, 0x0a, 0x1d, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, - 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, - 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x08, 0x12, 0x1c, 0x0a, 0x18, 0x46, 0x4c, 0x49, 0x47, 0x48, - 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x43, 0x41, 0x4e, - 0x43, 0x45, 0x4c, 0x10, 0x09, 0x12, 0x27, 0x0a, 0x23, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, - 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, - 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x4f, 0x55, 0x54, 0x10, 0x64, 0x12, 0x29, - 0x0a, 0x25, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, - 0x56, 0x45, 0x52, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, - 0x54, 0x49, 0x4d, 0x45, 0x4f, 0x55, 0x54, 0x10, 0x65, 0x12, 0x14, 0x0a, 0x0f, 0x53, 0x51, 0x4c, - 0x5f, 0x44, 0x44, 0x4c, 0x5f, 0x43, 0x41, 0x54, 0x41, 0x4c, 0x4f, 0x47, 0x10, 0xf4, 0x03, 0x12, - 0x13, 0x0a, 0x0e, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x44, 0x4c, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, - 0x41, 0x10, 0xf5, 0x03, 0x12, 0x12, 0x0a, 0x0d, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x44, 0x4c, 0x5f, - 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, 0xf6, 0x03, 0x12, 0x18, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, + 0x45, 0x52, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x54, 0x52, 0x41, 0x49, 0x54, 0x10, 0x05, 0x12, 0x2b, + 0x0a, 0x27, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, + 0x56, 0x45, 0x52, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x54, 0x52, 0x41, 0x49, 0x54, 0x5f, 0x4d, 0x49, + 0x4e, 0x5f, 0x56, 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x06, 0x12, 0x2b, 0x0a, 0x27, 0x46, + 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, + 0x5f, 0x53, 0x55, 0x42, 0x53, 0x54, 0x52, 0x41, 0x49, 0x54, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x56, + 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x07, 0x12, 0x21, 0x0a, 0x1d, 0x46, 0x4c, 0x49, 0x47, + 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x54, 0x52, + 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x08, 0x12, 0x1c, 0x0a, 0x18, 0x46, + 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, + 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x10, 0x09, 0x12, 0x24, 0x0a, 0x20, 0x46, 0x4c, 0x49, + 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x42, + 0x55, 0x4c, 0x4b, 0x5f, 0x49, 0x4e, 0x47, 0x45, 0x53, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x0a, 0x12, + 0x33, 0x0a, 0x2f, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, + 0x52, 0x56, 0x45, 0x52, 0x5f, 0x49, 0x4e, 0x47, 0x45, 0x53, 0x54, 0x5f, 0x54, 0x52, 0x41, 0x4e, + 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, + 0x45, 0x44, 0x10, 0x0b, 0x12, 0x27, 0x0a, 0x23, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, + 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x4d, + 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x4f, 0x55, 0x54, 0x10, 0x64, 0x12, 0x29, 0x0a, + 0x25, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, + 0x45, 0x52, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, + 0x49, 0x4d, 0x45, 0x4f, 0x55, 0x54, 0x10, 0x65, 0x12, 0x14, 0x0a, 0x0f, 0x53, 0x51, 0x4c, 0x5f, + 0x44, 0x44, 0x4c, 0x5f, 0x43, 0x41, 0x54, 0x41, 0x4c, 0x4f, 0x47, 0x10, 0xf4, 0x03, 0x12, 0x13, + 0x0a, 0x0e, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x44, 0x4c, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, + 0x10, 0xf5, 0x03, 0x12, 0x12, 0x0a, 0x0d, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x44, 0x4c, 0x5f, 0x54, + 0x41, 0x42, 0x4c, 0x45, 0x10, 0xf6, 0x03, 0x12, 0x18, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, 0x49, + 0x44, 0x45, 0x4e, 0x54, 0x49, 0x46, 0x49, 0x45, 0x52, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x10, 0xf7, + 0x03, 0x12, 0x1e, 0x0a, 0x19, 0x53, 0x51, 0x4c, 0x5f, 0x49, 0x44, 0x45, 0x4e, 0x54, 0x49, 0x46, + 0x49, 0x45, 0x52, 0x5f, 0x51, 0x55, 0x4f, 0x54, 0x45, 0x5f, 0x43, 0x48, 0x41, 0x52, 0x10, 0xf8, + 0x03, 0x12, 0x1f, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x51, 0x55, 0x4f, 0x54, 0x45, 0x44, 0x5f, 0x49, 0x44, 0x45, 0x4e, 0x54, 0x49, 0x46, 0x49, 0x45, 0x52, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x10, - 0xf7, 0x03, 0x12, 0x1e, 0x0a, 0x19, 0x53, 0x51, 0x4c, 0x5f, 0x49, 0x44, 0x45, 0x4e, 0x54, 0x49, - 0x46, 0x49, 0x45, 0x52, 0x5f, 0x51, 0x55, 0x4f, 0x54, 0x45, 0x5f, 0x43, 0x48, 0x41, 0x52, 0x10, - 0xf8, 0x03, 0x12, 0x1f, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x51, 0x55, 0x4f, 0x54, 0x45, 0x44, - 0x5f, 0x49, 0x44, 0x45, 0x4e, 0x54, 0x49, 0x46, 0x49, 0x45, 0x52, 0x5f, 0x43, 0x41, 0x53, 0x45, - 0x10, 0xf9, 0x03, 0x12, 0x22, 0x0a, 0x1d, 0x53, 0x51, 0x4c, 0x5f, 0x41, 0x4c, 0x4c, 0x5f, 0x54, - 0x41, 0x42, 0x4c, 0x45, 0x53, 0x5f, 0x41, 0x52, 0x45, 0x5f, 0x53, 0x45, 0x4c, 0x45, 0x43, 0x54, - 0x41, 0x42, 0x4c, 0x45, 0x10, 0xfa, 0x03, 0x12, 0x16, 0x0a, 0x11, 0x53, 0x51, 0x4c, 0x5f, 0x4e, - 0x55, 0x4c, 0x4c, 0x5f, 0x4f, 0x52, 0x44, 0x45, 0x52, 0x49, 0x4e, 0x47, 0x10, 0xfb, 0x03, 0x12, - 0x11, 0x0a, 0x0c, 0x53, 0x51, 0x4c, 0x5f, 0x4b, 0x45, 0x59, 0x57, 0x4f, 0x52, 0x44, 0x53, 0x10, - 0xfc, 0x03, 0x12, 0x1a, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, 0x4e, 0x55, 0x4d, 0x45, 0x52, 0x49, - 0x43, 0x5f, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0xfd, 0x03, 0x12, 0x19, - 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x46, 0x55, 0x4e, - 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0xfe, 0x03, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x51, 0x4c, - 0x5f, 0x53, 0x59, 0x53, 0x54, 0x45, 0x4d, 0x5f, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, - 0x53, 0x10, 0xff, 0x03, 0x12, 0x1b, 0x0a, 0x16, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x41, 0x54, 0x45, - 0x54, 0x49, 0x4d, 0x45, 0x5f, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x80, - 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x41, 0x52, 0x43, 0x48, 0x5f, - 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x45, 0x53, 0x43, 0x41, 0x50, 0x45, 0x10, 0x81, 0x04, - 0x12, 0x1e, 0x0a, 0x19, 0x53, 0x51, 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x52, 0x41, 0x5f, 0x4e, 0x41, - 0x4d, 0x45, 0x5f, 0x43, 0x48, 0x41, 0x52, 0x41, 0x43, 0x54, 0x45, 0x52, 0x53, 0x10, 0x82, 0x04, - 0x12, 0x21, 0x0a, 0x1c, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, - 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x5f, 0x41, 0x4c, 0x49, 0x41, 0x53, 0x49, 0x4e, 0x47, - 0x10, 0x83, 0x04, 0x12, 0x1f, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x5f, - 0x50, 0x4c, 0x55, 0x53, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x5f, 0x49, 0x53, 0x5f, 0x4e, 0x55, 0x4c, - 0x4c, 0x10, 0x84, 0x04, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, - 0x4f, 0x52, 0x54, 0x53, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x10, 0x85, 0x04, 0x12, - 0x29, 0x0a, 0x24, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, - 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x43, 0x4f, 0x52, 0x52, 0x45, 0x4c, 0x41, 0x54, 0x49, 0x4f, - 0x4e, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x10, 0x86, 0x04, 0x12, 0x33, 0x0a, 0x2e, 0x53, 0x51, - 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, 0x44, 0x49, 0x46, 0x46, 0x45, - 0x52, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x43, 0x4f, 0x52, 0x52, 0x45, - 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x10, 0x87, 0x04, 0x12, - 0x29, 0x0a, 0x24, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, - 0x45, 0x58, 0x50, 0x52, 0x45, 0x53, 0x53, 0x49, 0x4f, 0x4e, 0x53, 0x5f, 0x49, 0x4e, 0x5f, 0x4f, - 0x52, 0x44, 0x45, 0x52, 0x5f, 0x42, 0x59, 0x10, 0x88, 0x04, 0x12, 0x24, 0x0a, 0x1f, 0x53, 0x51, - 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, 0x4f, 0x52, 0x44, 0x45, 0x52, - 0x5f, 0x42, 0x59, 0x5f, 0x55, 0x4e, 0x52, 0x45, 0x4c, 0x41, 0x54, 0x45, 0x44, 0x10, 0x89, 0x04, - 0x12, 0x1b, 0x0a, 0x16, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, - 0x44, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x42, 0x59, 0x10, 0x8a, 0x04, 0x12, 0x24, 0x0a, - 0x1f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, 0x4c, 0x49, - 0x4b, 0x45, 0x5f, 0x45, 0x53, 0x43, 0x41, 0x50, 0x45, 0x5f, 0x43, 0x4c, 0x41, 0x55, 0x53, 0x45, - 0x10, 0x8b, 0x04, 0x12, 0x26, 0x0a, 0x21, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, - 0x52, 0x54, 0x53, 0x5f, 0x4e, 0x4f, 0x4e, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x41, 0x42, 0x4c, 0x45, - 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x53, 0x10, 0x8c, 0x04, 0x12, 0x1a, 0x0a, 0x15, 0x53, - 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x47, 0x52, 0x41, - 0x4d, 0x4d, 0x41, 0x52, 0x10, 0x8d, 0x04, 0x12, 0x1f, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x41, - 0x4e, 0x53, 0x49, 0x39, 0x32, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, - 0x4c, 0x45, 0x56, 0x45, 0x4c, 0x10, 0x8e, 0x04, 0x12, 0x30, 0x0a, 0x2b, 0x53, 0x51, 0x4c, 0x5f, - 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x47, 0x52, 0x49, - 0x54, 0x59, 0x5f, 0x45, 0x4e, 0x48, 0x41, 0x4e, 0x43, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x46, - 0x41, 0x43, 0x49, 0x4c, 0x49, 0x54, 0x59, 0x10, 0x8f, 0x04, 0x12, 0x22, 0x0a, 0x1d, 0x53, 0x51, - 0x4c, 0x5f, 0x4f, 0x55, 0x54, 0x45, 0x52, 0x5f, 0x4a, 0x4f, 0x49, 0x4e, 0x53, 0x5f, 0x53, 0x55, - 0x50, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x4c, 0x45, 0x56, 0x45, 0x4c, 0x10, 0x90, 0x04, 0x12, 0x14, - 0x0a, 0x0f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x54, 0x45, 0x52, - 0x4d, 0x10, 0x91, 0x04, 0x12, 0x17, 0x0a, 0x12, 0x53, 0x51, 0x4c, 0x5f, 0x50, 0x52, 0x4f, 0x43, - 0x45, 0x44, 0x55, 0x52, 0x45, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x10, 0x92, 0x04, 0x12, 0x15, 0x0a, - 0x10, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x54, 0x41, 0x4c, 0x4f, 0x47, 0x5f, 0x54, 0x45, 0x52, - 0x4d, 0x10, 0x93, 0x04, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x54, 0x41, - 0x4c, 0x4f, 0x47, 0x5f, 0x41, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x10, 0x94, 0x04, 0x12, - 0x22, 0x0a, 0x1d, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x53, 0x5f, 0x53, - 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, - 0x10, 0x95, 0x04, 0x12, 0x23, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x54, 0x41, 0x4c, - 0x4f, 0x47, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x41, 0x43, - 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x96, 0x04, 0x12, 0x26, 0x0a, 0x21, 0x53, 0x51, 0x4c, 0x5f, - 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x50, 0x4f, 0x53, 0x49, 0x54, 0x49, - 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x53, 0x10, 0x97, 0x04, - 0x12, 0x24, 0x0a, 0x1f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x4c, 0x45, 0x43, 0x54, 0x5f, 0x46, - 0x4f, 0x52, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, - 0x54, 0x45, 0x44, 0x10, 0x98, 0x04, 0x12, 0x24, 0x0a, 0x1f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x54, - 0x4f, 0x52, 0x45, 0x44, 0x5f, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x44, 0x55, 0x52, 0x45, 0x53, 0x5f, - 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0x99, 0x04, 0x12, 0x1d, 0x0a, 0x18, - 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x53, 0x55, - 0x42, 0x51, 0x55, 0x45, 0x52, 0x49, 0x45, 0x53, 0x10, 0x9a, 0x04, 0x12, 0x28, 0x0a, 0x23, 0x53, - 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x52, 0x52, 0x45, 0x4c, 0x41, 0x54, 0x45, 0x44, 0x5f, 0x53, 0x55, - 0x42, 0x51, 0x55, 0x45, 0x52, 0x49, 0x45, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, - 0x45, 0x44, 0x10, 0x9b, 0x04, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, - 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x55, 0x4e, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x9c, 0x04, - 0x12, 0x22, 0x0a, 0x1d, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x42, 0x49, 0x4e, 0x41, - 0x52, 0x59, 0x5f, 0x4c, 0x49, 0x54, 0x45, 0x52, 0x41, 0x4c, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, - 0x48, 0x10, 0x9d, 0x04, 0x12, 0x20, 0x0a, 0x1b, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, - 0x43, 0x48, 0x41, 0x52, 0x5f, 0x4c, 0x49, 0x54, 0x45, 0x52, 0x41, 0x4c, 0x5f, 0x4c, 0x45, 0x4e, - 0x47, 0x54, 0x48, 0x10, 0x9e, 0x04, 0x12, 0x1f, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, - 0x58, 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x5f, 0x4c, 0x45, - 0x4e, 0x47, 0x54, 0x48, 0x10, 0x9f, 0x04, 0x12, 0x20, 0x0a, 0x1b, 0x53, 0x51, 0x4c, 0x5f, 0x4d, - 0x41, 0x58, 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x53, 0x5f, 0x49, 0x4e, 0x5f, 0x47, 0x52, - 0x4f, 0x55, 0x50, 0x5f, 0x42, 0x59, 0x10, 0xa0, 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, 0x4c, + 0xf9, 0x03, 0x12, 0x22, 0x0a, 0x1d, 0x53, 0x51, 0x4c, 0x5f, 0x41, 0x4c, 0x4c, 0x5f, 0x54, 0x41, + 0x42, 0x4c, 0x45, 0x53, 0x5f, 0x41, 0x52, 0x45, 0x5f, 0x53, 0x45, 0x4c, 0x45, 0x43, 0x54, 0x41, + 0x42, 0x4c, 0x45, 0x10, 0xfa, 0x03, 0x12, 0x16, 0x0a, 0x11, 0x53, 0x51, 0x4c, 0x5f, 0x4e, 0x55, + 0x4c, 0x4c, 0x5f, 0x4f, 0x52, 0x44, 0x45, 0x52, 0x49, 0x4e, 0x47, 0x10, 0xfb, 0x03, 0x12, 0x11, + 0x0a, 0x0c, 0x53, 0x51, 0x4c, 0x5f, 0x4b, 0x45, 0x59, 0x57, 0x4f, 0x52, 0x44, 0x53, 0x10, 0xfc, + 0x03, 0x12, 0x1a, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, 0x4e, 0x55, 0x4d, 0x45, 0x52, 0x49, 0x43, + 0x5f, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0xfd, 0x03, 0x12, 0x19, 0x0a, + 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x46, 0x55, 0x4e, 0x43, + 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0xfe, 0x03, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, + 0x53, 0x59, 0x53, 0x54, 0x45, 0x4d, 0x5f, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, + 0x10, 0xff, 0x03, 0x12, 0x1b, 0x0a, 0x16, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x41, 0x54, 0x45, 0x54, + 0x49, 0x4d, 0x45, 0x5f, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x80, 0x04, + 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x41, 0x52, 0x43, 0x48, 0x5f, 0x53, + 0x54, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x45, 0x53, 0x43, 0x41, 0x50, 0x45, 0x10, 0x81, 0x04, 0x12, + 0x1e, 0x0a, 0x19, 0x53, 0x51, 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x52, 0x41, 0x5f, 0x4e, 0x41, 0x4d, + 0x45, 0x5f, 0x43, 0x48, 0x41, 0x52, 0x41, 0x43, 0x54, 0x45, 0x52, 0x53, 0x10, 0x82, 0x04, 0x12, + 0x21, 0x0a, 0x1c, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, + 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x5f, 0x41, 0x4c, 0x49, 0x41, 0x53, 0x49, 0x4e, 0x47, 0x10, + 0x83, 0x04, 0x12, 0x1f, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x5f, 0x50, + 0x4c, 0x55, 0x53, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x5f, 0x49, 0x53, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, + 0x10, 0x84, 0x04, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, + 0x52, 0x54, 0x53, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x10, 0x85, 0x04, 0x12, 0x29, + 0x0a, 0x24, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, 0x54, + 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x43, 0x4f, 0x52, 0x52, 0x45, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, + 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x10, 0x86, 0x04, 0x12, 0x33, 0x0a, 0x2e, 0x53, 0x51, 0x4c, + 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, 0x44, 0x49, 0x46, 0x46, 0x45, 0x52, + 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x43, 0x4f, 0x52, 0x52, 0x45, 0x4c, + 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x10, 0x87, 0x04, 0x12, 0x29, + 0x0a, 0x24, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, 0x45, + 0x58, 0x50, 0x52, 0x45, 0x53, 0x53, 0x49, 0x4f, 0x4e, 0x53, 0x5f, 0x49, 0x4e, 0x5f, 0x4f, 0x52, + 0x44, 0x45, 0x52, 0x5f, 0x42, 0x59, 0x10, 0x88, 0x04, 0x12, 0x24, 0x0a, 0x1f, 0x53, 0x51, 0x4c, + 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, 0x4f, 0x52, 0x44, 0x45, 0x52, 0x5f, + 0x42, 0x59, 0x5f, 0x55, 0x4e, 0x52, 0x45, 0x4c, 0x41, 0x54, 0x45, 0x44, 0x10, 0x89, 0x04, 0x12, + 0x1b, 0x0a, 0x16, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, + 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x42, 0x59, 0x10, 0x8a, 0x04, 0x12, 0x24, 0x0a, 0x1f, + 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, 0x4c, 0x49, 0x4b, + 0x45, 0x5f, 0x45, 0x53, 0x43, 0x41, 0x50, 0x45, 0x5f, 0x43, 0x4c, 0x41, 0x55, 0x53, 0x45, 0x10, + 0x8b, 0x04, 0x12, 0x26, 0x0a, 0x21, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, + 0x54, 0x53, 0x5f, 0x4e, 0x4f, 0x4e, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x5f, + 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x53, 0x10, 0x8c, 0x04, 0x12, 0x1a, 0x0a, 0x15, 0x53, 0x51, + 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x47, 0x52, 0x41, 0x4d, + 0x4d, 0x41, 0x52, 0x10, 0x8d, 0x04, 0x12, 0x1f, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x41, 0x4e, + 0x53, 0x49, 0x39, 0x32, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x4c, + 0x45, 0x56, 0x45, 0x4c, 0x10, 0x8e, 0x04, 0x12, 0x30, 0x0a, 0x2b, 0x53, 0x51, 0x4c, 0x5f, 0x53, + 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x47, 0x52, 0x49, 0x54, + 0x59, 0x5f, 0x45, 0x4e, 0x48, 0x41, 0x4e, 0x43, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x46, 0x41, + 0x43, 0x49, 0x4c, 0x49, 0x54, 0x59, 0x10, 0x8f, 0x04, 0x12, 0x22, 0x0a, 0x1d, 0x53, 0x51, 0x4c, + 0x5f, 0x4f, 0x55, 0x54, 0x45, 0x52, 0x5f, 0x4a, 0x4f, 0x49, 0x4e, 0x53, 0x5f, 0x53, 0x55, 0x50, + 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x4c, 0x45, 0x56, 0x45, 0x4c, 0x10, 0x90, 0x04, 0x12, 0x14, 0x0a, + 0x0f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x54, 0x45, 0x52, 0x4d, + 0x10, 0x91, 0x04, 0x12, 0x17, 0x0a, 0x12, 0x53, 0x51, 0x4c, 0x5f, 0x50, 0x52, 0x4f, 0x43, 0x45, + 0x44, 0x55, 0x52, 0x45, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x10, 0x92, 0x04, 0x12, 0x15, 0x0a, 0x10, + 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x54, 0x41, 0x4c, 0x4f, 0x47, 0x5f, 0x54, 0x45, 0x52, 0x4d, + 0x10, 0x93, 0x04, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x54, 0x41, 0x4c, + 0x4f, 0x47, 0x5f, 0x41, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x10, 0x94, 0x04, 0x12, 0x22, + 0x0a, 0x1d, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x53, 0x5f, 0x53, 0x55, + 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, + 0x95, 0x04, 0x12, 0x23, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x54, 0x41, 0x4c, 0x4f, + 0x47, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x41, 0x43, 0x54, + 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x96, 0x04, 0x12, 0x26, 0x0a, 0x21, 0x53, 0x51, 0x4c, 0x5f, 0x53, + 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x50, 0x4f, 0x53, 0x49, 0x54, 0x49, 0x4f, + 0x4e, 0x45, 0x44, 0x5f, 0x43, 0x4f, 0x4d, 0x4d, 0x41, 0x4e, 0x44, 0x53, 0x10, 0x97, 0x04, 0x12, + 0x24, 0x0a, 0x1f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x4c, 0x45, 0x43, 0x54, 0x5f, 0x46, 0x4f, + 0x52, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, + 0x45, 0x44, 0x10, 0x98, 0x04, 0x12, 0x24, 0x0a, 0x1f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x54, 0x4f, + 0x52, 0x45, 0x44, 0x5f, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x44, 0x55, 0x52, 0x45, 0x53, 0x5f, 0x53, + 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0x99, 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, + 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x53, 0x55, 0x42, + 0x51, 0x55, 0x45, 0x52, 0x49, 0x45, 0x53, 0x10, 0x9a, 0x04, 0x12, 0x28, 0x0a, 0x23, 0x53, 0x51, + 0x4c, 0x5f, 0x43, 0x4f, 0x52, 0x52, 0x45, 0x4c, 0x41, 0x54, 0x45, 0x44, 0x5f, 0x53, 0x55, 0x42, + 0x51, 0x55, 0x45, 0x52, 0x49, 0x45, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, + 0x44, 0x10, 0x9b, 0x04, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, + 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x55, 0x4e, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x9c, 0x04, 0x12, + 0x22, 0x0a, 0x1d, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x42, 0x49, 0x4e, 0x41, 0x52, + 0x59, 0x5f, 0x4c, 0x49, 0x54, 0x45, 0x52, 0x41, 0x4c, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, + 0x10, 0x9d, 0x04, 0x12, 0x20, 0x0a, 0x1b, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x43, + 0x48, 0x41, 0x52, 0x5f, 0x4c, 0x49, 0x54, 0x45, 0x52, 0x41, 0x4c, 0x5f, 0x4c, 0x45, 0x4e, 0x47, + 0x54, 0x48, 0x10, 0x9e, 0x04, 0x12, 0x1f, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, + 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x5f, 0x4c, 0x45, 0x4e, + 0x47, 0x54, 0x48, 0x10, 0x9f, 0x04, 0x12, 0x20, 0x0a, 0x1b, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, + 0x58, 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x53, 0x5f, 0x49, 0x4e, 0x5f, 0x47, 0x52, 0x4f, + 0x55, 0x50, 0x5f, 0x42, 0x59, 0x10, 0xa0, 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, 0x4c, 0x5f, + 0x4d, 0x41, 0x58, 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x53, 0x5f, 0x49, 0x4e, 0x5f, 0x49, + 0x4e, 0x44, 0x45, 0x58, 0x10, 0xa1, 0x04, 0x12, 0x20, 0x0a, 0x1b, 0x53, 0x51, 0x4c, 0x5f, 0x4d, + 0x41, 0x58, 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x53, 0x5f, 0x49, 0x4e, 0x5f, 0x4f, 0x52, + 0x44, 0x45, 0x52, 0x5f, 0x42, 0x59, 0x10, 0xa2, 0x04, 0x12, 0x1e, 0x0a, 0x19, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x53, 0x5f, 0x49, 0x4e, 0x5f, - 0x49, 0x4e, 0x44, 0x45, 0x58, 0x10, 0xa1, 0x04, 0x12, 0x20, 0x0a, 0x1b, 0x53, 0x51, 0x4c, 0x5f, - 0x4d, 0x41, 0x58, 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x53, 0x5f, 0x49, 0x4e, 0x5f, 0x4f, - 0x52, 0x44, 0x45, 0x52, 0x5f, 0x42, 0x59, 0x10, 0xa2, 0x04, 0x12, 0x1e, 0x0a, 0x19, 0x53, 0x51, - 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x53, 0x5f, 0x49, 0x4e, - 0x5f, 0x53, 0x45, 0x4c, 0x45, 0x43, 0x54, 0x10, 0xa3, 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, - 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x53, 0x5f, 0x49, 0x4e, - 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, 0xa4, 0x04, 0x12, 0x18, 0x0a, 0x13, 0x53, 0x51, 0x4c, - 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, - 0x10, 0xa5, 0x04, 0x12, 0x1f, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x43, - 0x55, 0x52, 0x53, 0x4f, 0x52, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, - 0x48, 0x10, 0xa6, 0x04, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, - 0x49, 0x4e, 0x44, 0x45, 0x58, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x10, 0xa7, 0x04, 0x12, - 0x1e, 0x0a, 0x19, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x42, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, - 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x10, 0xa8, 0x04, 0x12, - 0x22, 0x0a, 0x1d, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x50, 0x52, 0x4f, 0x43, 0x45, - 0x44, 0x55, 0x52, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, - 0x10, 0xa9, 0x04, 0x12, 0x20, 0x0a, 0x1b, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x43, - 0x41, 0x54, 0x41, 0x4c, 0x4f, 0x47, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x5f, 0x4c, 0x45, 0x4e, 0x47, - 0x54, 0x48, 0x10, 0xaa, 0x04, 0x12, 0x15, 0x0a, 0x10, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, - 0x5f, 0x52, 0x4f, 0x57, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x10, 0xab, 0x04, 0x12, 0x24, 0x0a, 0x1f, - 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x52, 0x4f, 0x57, 0x5f, 0x53, 0x49, 0x5a, 0x45, - 0x5f, 0x49, 0x4e, 0x43, 0x4c, 0x55, 0x44, 0x45, 0x53, 0x5f, 0x42, 0x4c, 0x4f, 0x42, 0x53, 0x10, - 0xac, 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x53, 0x54, - 0x41, 0x54, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x10, 0xad, - 0x04, 0x12, 0x17, 0x0a, 0x12, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x53, 0x54, 0x41, - 0x54, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x53, 0x10, 0xae, 0x04, 0x12, 0x1e, 0x0a, 0x19, 0x53, 0x51, - 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, - 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x10, 0xaf, 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, - 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x53, 0x5f, 0x49, 0x4e, 0x5f, - 0x53, 0x45, 0x4c, 0x45, 0x43, 0x54, 0x10, 0xb0, 0x04, 0x12, 0x1c, 0x0a, 0x17, 0x53, 0x51, 0x4c, - 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x55, 0x53, 0x45, 0x52, 0x4e, 0x41, 0x4d, 0x45, 0x5f, 0x4c, 0x45, - 0x4e, 0x47, 0x54, 0x48, 0x10, 0xb1, 0x04, 0x12, 0x26, 0x0a, 0x21, 0x53, 0x51, 0x4c, 0x5f, 0x44, - 0x45, 0x46, 0x41, 0x55, 0x4c, 0x54, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, - 0x4f, 0x4e, 0x5f, 0x49, 0x53, 0x4f, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0xb2, 0x04, 0x12, - 0x1f, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, - 0x4f, 0x4e, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0xb3, 0x04, - 0x12, 0x30, 0x0a, 0x2b, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, - 0x44, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x5f, 0x49, - 0x53, 0x4f, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4c, 0x45, 0x56, 0x45, 0x4c, 0x53, 0x10, - 0xb4, 0x04, 0x12, 0x32, 0x0a, 0x2d, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x44, - 0x45, 0x46, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x53, - 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x4f, 0x4d, - 0x4d, 0x49, 0x54, 0x10, 0xb5, 0x04, 0x12, 0x31, 0x0a, 0x2c, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x41, - 0x54, 0x41, 0x5f, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x5f, 0x49, - 0x4e, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x5f, 0x49, - 0x47, 0x4e, 0x4f, 0x52, 0x45, 0x44, 0x10, 0xb6, 0x04, 0x12, 0x23, 0x0a, 0x1e, 0x53, 0x51, 0x4c, - 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, - 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x53, 0x10, 0xb7, 0x04, 0x12, 0x3b, - 0x0a, 0x36, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, - 0x43, 0x4f, 0x4e, 0x43, 0x55, 0x52, 0x52, 0x45, 0x4e, 0x43, 0x49, 0x45, 0x53, 0x5f, 0x46, 0x4f, - 0x52, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x55, 0x4e, 0x53, - 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0xb8, 0x04, 0x12, 0x3c, 0x0a, 0x37, 0x53, - 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x43, 0x4f, 0x4e, - 0x43, 0x55, 0x52, 0x52, 0x45, 0x4e, 0x43, 0x49, 0x45, 0x53, 0x5f, 0x46, 0x4f, 0x52, 0x5f, 0x52, - 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x46, 0x4f, 0x52, 0x57, 0x41, 0x52, - 0x44, 0x5f, 0x4f, 0x4e, 0x4c, 0x59, 0x10, 0xb9, 0x04, 0x12, 0x40, 0x0a, 0x3b, 0x53, 0x51, 0x4c, - 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x43, 0x4f, 0x4e, 0x43, 0x55, - 0x52, 0x52, 0x45, 0x4e, 0x43, 0x49, 0x45, 0x53, 0x5f, 0x46, 0x4f, 0x52, 0x5f, 0x52, 0x45, 0x53, - 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x53, 0x43, 0x52, 0x4f, 0x4c, 0x4c, 0x5f, 0x53, - 0x45, 0x4e, 0x53, 0x49, 0x54, 0x49, 0x56, 0x45, 0x10, 0xba, 0x04, 0x12, 0x42, 0x0a, 0x3d, 0x53, - 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x43, 0x4f, 0x4e, - 0x43, 0x55, 0x52, 0x52, 0x45, 0x4e, 0x43, 0x49, 0x45, 0x53, 0x5f, 0x46, 0x4f, 0x52, 0x5f, 0x52, - 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x53, 0x43, 0x52, 0x4f, 0x4c, 0x4c, - 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x4e, 0x53, 0x49, 0x54, 0x49, 0x56, 0x45, 0x10, 0xbb, 0x04, 0x12, - 0x20, 0x0a, 0x1b, 0x53, 0x51, 0x4c, 0x5f, 0x42, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x55, 0x50, 0x44, - 0x41, 0x54, 0x45, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0xbc, - 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x41, 0x56, 0x45, 0x50, 0x4f, 0x49, - 0x4e, 0x54, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0xbd, 0x04, - 0x12, 0x23, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x44, 0x5f, 0x50, 0x41, - 0x52, 0x41, 0x4d, 0x45, 0x54, 0x45, 0x52, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, - 0x45, 0x44, 0x10, 0xbe, 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, 0x4c, 0x5f, 0x4c, 0x4f, 0x43, - 0x41, 0x54, 0x4f, 0x52, 0x53, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x43, 0x4f, 0x50, - 0x59, 0x10, 0xbf, 0x04, 0x12, 0x35, 0x0a, 0x30, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x54, 0x4f, 0x52, - 0x45, 0x44, 0x5f, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x5f, 0x55, 0x53, 0x49, - 0x4e, 0x47, 0x5f, 0x43, 0x41, 0x4c, 0x4c, 0x5f, 0x53, 0x59, 0x4e, 0x54, 0x41, 0x58, 0x5f, 0x53, - 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0xc0, 0x04, 0x2a, 0x91, 0x01, 0x0a, 0x17, - 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x72, 0x61, 0x6e, - 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, 0x53, - 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, - 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x29, 0x0a, 0x25, 0x53, - 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x54, 0x52, 0x41, - 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, - 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x12, 0x27, 0x0a, 0x23, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, + 0x53, 0x45, 0x4c, 0x45, 0x43, 0x54, 0x10, 0xa3, 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, 0x4c, + 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x43, 0x4f, 0x4c, 0x55, 0x4d, 0x4e, 0x53, 0x5f, 0x49, 0x4e, 0x5f, + 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, 0xa4, 0x04, 0x12, 0x18, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, + 0x4d, 0x41, 0x58, 0x5f, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, + 0xa5, 0x04, 0x12, 0x1f, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x43, 0x55, + 0x52, 0x53, 0x4f, 0x52, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, + 0x10, 0xa6, 0x04, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x49, + 0x4e, 0x44, 0x45, 0x58, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x10, 0xa7, 0x04, 0x12, 0x1e, + 0x0a, 0x19, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x42, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, + 0x4e, 0x41, 0x4d, 0x45, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x10, 0xa8, 0x04, 0x12, 0x22, + 0x0a, 0x1d, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x44, + 0x55, 0x52, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x10, + 0xa9, 0x04, 0x12, 0x20, 0x0a, 0x1b, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x43, 0x41, + 0x54, 0x41, 0x4c, 0x4f, 0x47, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, + 0x48, 0x10, 0xaa, 0x04, 0x12, 0x15, 0x0a, 0x10, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, + 0x52, 0x4f, 0x57, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x10, 0xab, 0x04, 0x12, 0x24, 0x0a, 0x1f, 0x53, + 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x52, 0x4f, 0x57, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x5f, + 0x49, 0x4e, 0x43, 0x4c, 0x55, 0x44, 0x45, 0x53, 0x5f, 0x42, 0x4c, 0x4f, 0x42, 0x53, 0x10, 0xac, + 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x53, 0x54, 0x41, + 0x54, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x10, 0xad, 0x04, + 0x12, 0x17, 0x0a, 0x12, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x53, 0x54, 0x41, 0x54, + 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x53, 0x10, 0xae, 0x04, 0x12, 0x1e, 0x0a, 0x19, 0x53, 0x51, 0x4c, + 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x5f, + 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x10, 0xaf, 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, 0x4c, + 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x53, 0x5f, 0x49, 0x4e, 0x5f, 0x53, + 0x45, 0x4c, 0x45, 0x43, 0x54, 0x10, 0xb0, 0x04, 0x12, 0x1c, 0x0a, 0x17, 0x53, 0x51, 0x4c, 0x5f, + 0x4d, 0x41, 0x58, 0x5f, 0x55, 0x53, 0x45, 0x52, 0x4e, 0x41, 0x4d, 0x45, 0x5f, 0x4c, 0x45, 0x4e, + 0x47, 0x54, 0x48, 0x10, 0xb1, 0x04, 0x12, 0x26, 0x0a, 0x21, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x45, + 0x46, 0x41, 0x55, 0x4c, 0x54, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, + 0x4e, 0x5f, 0x49, 0x53, 0x4f, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0xb2, 0x04, 0x12, 0x1f, + 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, + 0x4e, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0xb3, 0x04, 0x12, + 0x30, 0x0a, 0x2b, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, + 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x5f, 0x49, 0x53, + 0x4f, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4c, 0x45, 0x56, 0x45, 0x4c, 0x53, 0x10, 0xb4, + 0x04, 0x12, 0x32, 0x0a, 0x2d, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x44, 0x45, + 0x46, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x55, 0x53, 0x45, 0x53, 0x5f, + 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x4f, 0x4d, 0x4d, + 0x49, 0x54, 0x10, 0xb5, 0x04, 0x12, 0x31, 0x0a, 0x2c, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x41, 0x54, + 0x41, 0x5f, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x5f, 0x49, 0x4e, + 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x5f, 0x49, 0x47, + 0x4e, 0x4f, 0x52, 0x45, 0x44, 0x10, 0xb6, 0x04, 0x12, 0x23, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, + 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, + 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x53, 0x10, 0xb7, 0x04, 0x12, 0x3b, 0x0a, + 0x36, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x43, + 0x4f, 0x4e, 0x43, 0x55, 0x52, 0x52, 0x45, 0x4e, 0x43, 0x49, 0x45, 0x53, 0x5f, 0x46, 0x4f, 0x52, + 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x55, 0x4e, 0x53, 0x50, + 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0xb8, 0x04, 0x12, 0x3c, 0x0a, 0x37, 0x53, 0x51, + 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x43, 0x4f, 0x4e, 0x43, + 0x55, 0x52, 0x52, 0x45, 0x4e, 0x43, 0x49, 0x45, 0x53, 0x5f, 0x46, 0x4f, 0x52, 0x5f, 0x52, 0x45, + 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x46, 0x4f, 0x52, 0x57, 0x41, 0x52, 0x44, + 0x5f, 0x4f, 0x4e, 0x4c, 0x59, 0x10, 0xb9, 0x04, 0x12, 0x40, 0x0a, 0x3b, 0x53, 0x51, 0x4c, 0x5f, + 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x43, 0x4f, 0x4e, 0x43, 0x55, 0x52, + 0x52, 0x45, 0x4e, 0x43, 0x49, 0x45, 0x53, 0x5f, 0x46, 0x4f, 0x52, 0x5f, 0x52, 0x45, 0x53, 0x55, + 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x53, 0x43, 0x52, 0x4f, 0x4c, 0x4c, 0x5f, 0x53, 0x45, + 0x4e, 0x53, 0x49, 0x54, 0x49, 0x56, 0x45, 0x10, 0xba, 0x04, 0x12, 0x42, 0x0a, 0x3d, 0x53, 0x51, + 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x43, 0x4f, 0x4e, 0x43, + 0x55, 0x52, 0x52, 0x45, 0x4e, 0x43, 0x49, 0x45, 0x53, 0x5f, 0x46, 0x4f, 0x52, 0x5f, 0x52, 0x45, + 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x53, 0x43, 0x52, 0x4f, 0x4c, 0x4c, 0x5f, + 0x49, 0x4e, 0x53, 0x45, 0x4e, 0x53, 0x49, 0x54, 0x49, 0x56, 0x45, 0x10, 0xbb, 0x04, 0x12, 0x20, + 0x0a, 0x1b, 0x53, 0x51, 0x4c, 0x5f, 0x42, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x55, 0x50, 0x44, 0x41, + 0x54, 0x45, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0xbc, 0x04, + 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x41, 0x56, 0x45, 0x50, 0x4f, 0x49, 0x4e, + 0x54, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0xbd, 0x04, 0x12, + 0x23, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x44, 0x5f, 0x50, 0x41, 0x52, + 0x41, 0x4d, 0x45, 0x54, 0x45, 0x52, 0x53, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, + 0x44, 0x10, 0xbe, 0x04, 0x12, 0x1d, 0x0a, 0x18, 0x53, 0x51, 0x4c, 0x5f, 0x4c, 0x4f, 0x43, 0x41, + 0x54, 0x4f, 0x52, 0x53, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x43, 0x4f, 0x50, 0x59, + 0x10, 0xbf, 0x04, 0x12, 0x35, 0x0a, 0x30, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x54, 0x4f, 0x52, 0x45, + 0x44, 0x5f, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x5f, 0x55, 0x53, 0x49, 0x4e, + 0x47, 0x5f, 0x43, 0x41, 0x4c, 0x4c, 0x5f, 0x53, 0x59, 0x4e, 0x54, 0x41, 0x58, 0x5f, 0x53, 0x55, + 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0xc0, 0x04, 0x2a, 0x91, 0x01, 0x0a, 0x17, 0x53, + 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x72, 0x61, 0x6e, 0x73, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, - 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x41, 0x56, 0x45, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x02, 0x2a, - 0xb2, 0x01, 0x0a, 0x1b, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, - 0x43, 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x12, - 0x20, 0x0a, 0x1c, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x5f, 0x53, 0x45, 0x4e, 0x53, - 0x49, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, - 0x00, 0x12, 0x29, 0x0a, 0x25, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x5f, 0x53, 0x45, - 0x4e, 0x53, 0x49, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x5f, 0x49, - 0x4e, 0x53, 0x45, 0x4e, 0x53, 0x49, 0x54, 0x49, 0x56, 0x45, 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, - 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x5f, 0x53, 0x45, 0x4e, 0x53, 0x49, 0x54, 0x49, - 0x56, 0x49, 0x54, 0x59, 0x5f, 0x55, 0x50, 0x50, 0x45, 0x52, 0x43, 0x41, 0x53, 0x45, 0x10, 0x02, - 0x12, 0x22, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x5f, 0x53, 0x45, 0x4e, - 0x53, 0x49, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x4c, 0x4f, 0x57, 0x45, 0x52, 0x43, 0x41, - 0x53, 0x45, 0x10, 0x03, 0x2a, 0x82, 0x01, 0x0a, 0x0f, 0x53, 0x71, 0x6c, 0x4e, 0x75, 0x6c, 0x6c, - 0x4f, 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x19, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, - 0x4e, 0x55, 0x4c, 0x4c, 0x53, 0x5f, 0x53, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x48, 0x49, 0x47, - 0x48, 0x10, 0x00, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x53, - 0x5f, 0x53, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x4c, 0x4f, 0x57, 0x10, 0x01, 0x12, 0x1d, 0x0a, - 0x19, 0x53, 0x51, 0x4c, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x53, 0x5f, 0x53, 0x4f, 0x52, 0x54, 0x45, - 0x44, 0x5f, 0x41, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x10, 0x02, 0x12, 0x1b, 0x0a, 0x17, + 0x49, 0x4f, 0x4e, 0x5f, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x29, 0x0a, 0x25, 0x53, 0x51, + 0x4c, 0x5f, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x54, 0x52, 0x41, 0x4e, + 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, + 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x12, 0x27, 0x0a, 0x23, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x50, + 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x53, 0x41, 0x56, 0x45, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x02, 0x2a, 0xb2, + 0x01, 0x0a, 0x1b, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x43, + 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x12, 0x20, + 0x0a, 0x1c, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x5f, 0x53, 0x45, 0x4e, 0x53, 0x49, + 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, + 0x12, 0x29, 0x0a, 0x25, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x5f, 0x53, 0x45, 0x4e, + 0x53, 0x49, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x5f, 0x49, 0x4e, + 0x53, 0x45, 0x4e, 0x53, 0x49, 0x54, 0x49, 0x56, 0x45, 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x53, + 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x5f, 0x53, 0x45, 0x4e, 0x53, 0x49, 0x54, 0x49, 0x56, + 0x49, 0x54, 0x59, 0x5f, 0x55, 0x50, 0x50, 0x45, 0x52, 0x43, 0x41, 0x53, 0x45, 0x10, 0x02, 0x12, + 0x22, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x5f, 0x53, 0x45, 0x4e, 0x53, + 0x49, 0x54, 0x49, 0x56, 0x49, 0x54, 0x59, 0x5f, 0x4c, 0x4f, 0x57, 0x45, 0x52, 0x43, 0x41, 0x53, + 0x45, 0x10, 0x03, 0x2a, 0x82, 0x01, 0x0a, 0x0f, 0x53, 0x71, 0x6c, 0x4e, 0x75, 0x6c, 0x6c, 0x4f, + 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x19, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, 0x4e, + 0x55, 0x4c, 0x4c, 0x53, 0x5f, 0x53, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x48, 0x49, 0x47, 0x48, + 0x10, 0x00, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x53, 0x5f, + 0x53, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x4c, 0x4f, 0x57, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x53, 0x51, 0x4c, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x53, 0x5f, 0x53, 0x4f, 0x52, 0x54, 0x45, 0x44, - 0x5f, 0x41, 0x54, 0x5f, 0x45, 0x4e, 0x44, 0x10, 0x03, 0x2a, 0x5e, 0x0a, 0x13, 0x53, 0x75, 0x70, - 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x71, 0x6c, 0x47, 0x72, 0x61, 0x6d, 0x6d, 0x61, 0x72, - 0x12, 0x17, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x49, 0x4e, 0x49, 0x4d, 0x55, 0x4d, 0x5f, - 0x47, 0x52, 0x41, 0x4d, 0x4d, 0x41, 0x52, 0x10, 0x00, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x51, 0x4c, - 0x5f, 0x43, 0x4f, 0x52, 0x45, 0x5f, 0x47, 0x52, 0x41, 0x4d, 0x4d, 0x41, 0x52, 0x10, 0x01, 0x12, - 0x18, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x4e, 0x44, 0x45, 0x44, 0x5f, - 0x47, 0x52, 0x41, 0x4d, 0x4d, 0x41, 0x52, 0x10, 0x02, 0x2a, 0x68, 0x0a, 0x1e, 0x53, 0x75, 0x70, - 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x41, 0x6e, 0x73, 0x69, 0x39, 0x32, 0x53, 0x71, 0x6c, 0x47, - 0x72, 0x61, 0x6d, 0x6d, 0x61, 0x72, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x14, 0x0a, 0x10, 0x41, - 0x4e, 0x53, 0x49, 0x39, 0x32, 0x5f, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x53, 0x51, 0x4c, 0x10, - 0x00, 0x12, 0x1b, 0x0a, 0x17, 0x41, 0x4e, 0x53, 0x49, 0x39, 0x32, 0x5f, 0x49, 0x4e, 0x54, 0x45, - 0x52, 0x4d, 0x45, 0x44, 0x49, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x51, 0x4c, 0x10, 0x01, 0x12, 0x13, - 0x0a, 0x0f, 0x41, 0x4e, 0x53, 0x49, 0x39, 0x32, 0x5f, 0x46, 0x55, 0x4c, 0x4c, 0x5f, 0x53, 0x51, - 0x4c, 0x10, 0x02, 0x2a, 0x6d, 0x0a, 0x19, 0x53, 0x71, 0x6c, 0x4f, 0x75, 0x74, 0x65, 0x72, 0x4a, - 0x6f, 0x69, 0x6e, 0x73, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, - 0x12, 0x19, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, 0x4a, 0x4f, 0x49, 0x4e, 0x53, 0x5f, 0x55, 0x4e, - 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1b, 0x0a, 0x17, 0x53, - 0x51, 0x4c, 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x45, 0x52, - 0x5f, 0x4a, 0x4f, 0x49, 0x4e, 0x53, 0x10, 0x01, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, - 0x46, 0x55, 0x4c, 0x4c, 0x5f, 0x4f, 0x55, 0x54, 0x45, 0x52, 0x5f, 0x4a, 0x4f, 0x49, 0x4e, 0x53, - 0x10, 0x02, 0x2a, 0x51, 0x0a, 0x13, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, - 0x65, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x12, 0x1a, 0x0a, 0x16, 0x53, 0x51, 0x4c, - 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x42, 0x59, 0x5f, 0x55, 0x4e, 0x52, 0x45, 0x4c, 0x41, - 0x54, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1e, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x47, 0x52, 0x4f, - 0x55, 0x50, 0x5f, 0x42, 0x59, 0x5f, 0x42, 0x45, 0x59, 0x4f, 0x4e, 0x44, 0x5f, 0x53, 0x45, 0x4c, - 0x45, 0x43, 0x54, 0x10, 0x01, 0x2a, 0x90, 0x01, 0x0a, 0x1a, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, - 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x22, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, 0x45, 0x4c, 0x45, 0x4d, - 0x45, 0x4e, 0x54, 0x5f, 0x49, 0x4e, 0x5f, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x44, 0x55, 0x52, 0x45, - 0x5f, 0x43, 0x41, 0x4c, 0x4c, 0x53, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x51, 0x4c, 0x5f, - 0x45, 0x4c, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x49, 0x4e, 0x5f, 0x49, 0x4e, 0x44, 0x45, 0x58, - 0x5f, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x01, 0x12, 0x28, - 0x0a, 0x24, 0x53, 0x51, 0x4c, 0x5f, 0x45, 0x4c, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x49, 0x4e, - 0x5f, 0x50, 0x52, 0x49, 0x56, 0x49, 0x4c, 0x45, 0x47, 0x45, 0x5f, 0x44, 0x45, 0x46, 0x49, 0x4e, - 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x02, 0x2a, 0x56, 0x0a, 0x1e, 0x53, 0x71, 0x6c, 0x53, - 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x65, 0x64, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x12, 0x19, 0x0a, 0x15, 0x53, 0x51, - 0x4c, 0x5f, 0x50, 0x4f, 0x53, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x44, 0x45, 0x4c, - 0x45, 0x54, 0x45, 0x10, 0x00, 0x12, 0x19, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, 0x50, 0x4f, 0x53, - 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x01, - 0x2a, 0x97, 0x01, 0x0a, 0x16, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, - 0x64, 0x53, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x69, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x1d, 0x53, + 0x5f, 0x41, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x10, 0x02, 0x12, 0x1b, 0x0a, 0x17, 0x53, + 0x51, 0x4c, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x53, 0x5f, 0x53, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, + 0x41, 0x54, 0x5f, 0x45, 0x4e, 0x44, 0x10, 0x03, 0x2a, 0x5e, 0x0a, 0x13, 0x53, 0x75, 0x70, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x71, 0x6c, 0x47, 0x72, 0x61, 0x6d, 0x6d, 0x61, 0x72, 0x12, + 0x17, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, 0x4d, 0x49, 0x4e, 0x49, 0x4d, 0x55, 0x4d, 0x5f, 0x47, + 0x52, 0x41, 0x4d, 0x4d, 0x41, 0x52, 0x10, 0x00, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x51, 0x4c, 0x5f, + 0x43, 0x4f, 0x52, 0x45, 0x5f, 0x47, 0x52, 0x41, 0x4d, 0x4d, 0x41, 0x52, 0x10, 0x01, 0x12, 0x18, + 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x45, 0x58, 0x54, 0x45, 0x4e, 0x44, 0x45, 0x44, 0x5f, 0x47, + 0x52, 0x41, 0x4d, 0x4d, 0x41, 0x52, 0x10, 0x02, 0x2a, 0x68, 0x0a, 0x1e, 0x53, 0x75, 0x70, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x41, 0x6e, 0x73, 0x69, 0x39, 0x32, 0x53, 0x71, 0x6c, 0x47, 0x72, + 0x61, 0x6d, 0x6d, 0x61, 0x72, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x14, 0x0a, 0x10, 0x41, 0x4e, + 0x53, 0x49, 0x39, 0x32, 0x5f, 0x45, 0x4e, 0x54, 0x52, 0x59, 0x5f, 0x53, 0x51, 0x4c, 0x10, 0x00, + 0x12, 0x1b, 0x0a, 0x17, 0x41, 0x4e, 0x53, 0x49, 0x39, 0x32, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, + 0x4d, 0x45, 0x44, 0x49, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x51, 0x4c, 0x10, 0x01, 0x12, 0x13, 0x0a, + 0x0f, 0x41, 0x4e, 0x53, 0x49, 0x39, 0x32, 0x5f, 0x46, 0x55, 0x4c, 0x4c, 0x5f, 0x53, 0x51, 0x4c, + 0x10, 0x02, 0x2a, 0x6d, 0x0a, 0x19, 0x53, 0x71, 0x6c, 0x4f, 0x75, 0x74, 0x65, 0x72, 0x4a, 0x6f, + 0x69, 0x6e, 0x73, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, + 0x19, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, 0x4a, 0x4f, 0x49, 0x4e, 0x53, 0x5f, 0x55, 0x4e, 0x53, + 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1b, 0x0a, 0x17, 0x53, 0x51, + 0x4c, 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x45, 0x52, 0x5f, + 0x4a, 0x4f, 0x49, 0x4e, 0x53, 0x10, 0x01, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x46, + 0x55, 0x4c, 0x4c, 0x5f, 0x4f, 0x55, 0x54, 0x45, 0x52, 0x5f, 0x4a, 0x4f, 0x49, 0x4e, 0x53, 0x10, + 0x02, 0x2a, 0x51, 0x0a, 0x13, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, + 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x12, 0x1a, 0x0a, 0x16, 0x53, 0x51, 0x4c, 0x5f, + 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x42, 0x59, 0x5f, 0x55, 0x4e, 0x52, 0x45, 0x4c, 0x41, 0x54, + 0x45, 0x44, 0x10, 0x00, 0x12, 0x1e, 0x0a, 0x1a, 0x53, 0x51, 0x4c, 0x5f, 0x47, 0x52, 0x4f, 0x55, + 0x50, 0x5f, 0x42, 0x59, 0x5f, 0x42, 0x45, 0x59, 0x4f, 0x4e, 0x44, 0x5f, 0x53, 0x45, 0x4c, 0x45, + 0x43, 0x54, 0x10, 0x01, 0x2a, 0x90, 0x01, 0x0a, 0x1a, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x12, 0x22, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, 0x45, 0x4c, 0x45, 0x4d, 0x45, + 0x4e, 0x54, 0x5f, 0x49, 0x4e, 0x5f, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x44, 0x55, 0x52, 0x45, 0x5f, + 0x43, 0x41, 0x4c, 0x4c, 0x53, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x51, 0x4c, 0x5f, 0x45, + 0x4c, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x49, 0x4e, 0x5f, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x5f, + 0x44, 0x45, 0x46, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x01, 0x12, 0x28, 0x0a, + 0x24, 0x53, 0x51, 0x4c, 0x5f, 0x45, 0x4c, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x49, 0x4e, 0x5f, + 0x50, 0x52, 0x49, 0x56, 0x49, 0x4c, 0x45, 0x47, 0x45, 0x5f, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x49, + 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x02, 0x2a, 0x56, 0x0a, 0x1e, 0x53, 0x71, 0x6c, 0x53, 0x75, + 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x65, + 0x64, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x12, 0x19, 0x0a, 0x15, 0x53, 0x51, 0x4c, + 0x5f, 0x50, 0x4f, 0x53, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x44, 0x45, 0x4c, 0x45, + 0x54, 0x45, 0x10, 0x00, 0x12, 0x19, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, 0x50, 0x4f, 0x53, 0x49, + 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x01, 0x2a, + 0x97, 0x01, 0x0a, 0x16, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, + 0x53, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x69, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x1d, 0x53, 0x51, + 0x4c, 0x5f, 0x53, 0x55, 0x42, 0x51, 0x55, 0x45, 0x52, 0x49, 0x45, 0x53, 0x5f, 0x49, 0x4e, 0x5f, + 0x43, 0x4f, 0x4d, 0x50, 0x41, 0x52, 0x49, 0x53, 0x4f, 0x4e, 0x53, 0x10, 0x00, 0x12, 0x1c, 0x0a, + 0x18, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x42, 0x51, 0x55, 0x45, 0x52, 0x49, 0x45, 0x53, 0x5f, + 0x49, 0x4e, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, 0x53, 0x10, 0x01, 0x12, 0x19, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x42, 0x51, 0x55, 0x45, 0x52, 0x49, 0x45, 0x53, 0x5f, 0x49, 0x4e, - 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x41, 0x52, 0x49, 0x53, 0x4f, 0x4e, 0x53, 0x10, 0x00, 0x12, 0x1c, - 0x0a, 0x18, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x42, 0x51, 0x55, 0x45, 0x52, 0x49, 0x45, 0x53, - 0x5f, 0x49, 0x4e, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, 0x53, 0x10, 0x01, 0x12, 0x19, 0x0a, 0x15, - 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, 0x42, 0x51, 0x55, 0x45, 0x52, 0x49, 0x45, 0x53, 0x5f, 0x49, - 0x4e, 0x5f, 0x49, 0x4e, 0x53, 0x10, 0x02, 0x12, 0x21, 0x0a, 0x1d, 0x53, 0x51, 0x4c, 0x5f, 0x53, - 0x55, 0x42, 0x51, 0x55, 0x45, 0x52, 0x49, 0x45, 0x53, 0x5f, 0x49, 0x4e, 0x5f, 0x51, 0x55, 0x41, - 0x4e, 0x54, 0x49, 0x46, 0x49, 0x45, 0x44, 0x53, 0x10, 0x03, 0x2a, 0x36, 0x0a, 0x12, 0x53, 0x71, - 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x55, 0x6e, 0x69, 0x6f, 0x6e, 0x73, - 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x51, 0x4c, 0x5f, 0x55, 0x4e, 0x49, 0x4f, 0x4e, 0x10, 0x00, 0x12, - 0x11, 0x0a, 0x0d, 0x53, 0x51, 0x4c, 0x5f, 0x55, 0x4e, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x4c, 0x4c, - 0x10, 0x01, 0x2a, 0xc9, 0x01, 0x0a, 0x1c, 0x53, 0x71, 0x6c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x73, 0x6f, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x65, - 0x76, 0x65, 0x6c, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, - 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x24, 0x0a, - 0x20, 0x53, 0x51, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, - 0x5f, 0x52, 0x45, 0x41, 0x44, 0x5f, 0x55, 0x4e, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x54, 0x45, - 0x44, 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, - 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x5f, 0x43, 0x4f, 0x4d, 0x4d, - 0x49, 0x54, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x51, 0x4c, 0x5f, 0x54, - 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x50, 0x45, 0x41, - 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x10, 0x03, 0x12, 0x20, 0x0a, 0x1c, + 0x5f, 0x49, 0x4e, 0x53, 0x10, 0x02, 0x12, 0x21, 0x0a, 0x1d, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x55, + 0x42, 0x51, 0x55, 0x45, 0x52, 0x49, 0x45, 0x53, 0x5f, 0x49, 0x4e, 0x5f, 0x51, 0x55, 0x41, 0x4e, + 0x54, 0x49, 0x46, 0x49, 0x45, 0x44, 0x53, 0x10, 0x03, 0x2a, 0x36, 0x0a, 0x12, 0x53, 0x71, 0x6c, + 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x55, 0x6e, 0x69, 0x6f, 0x6e, 0x73, 0x12, + 0x0d, 0x0a, 0x09, 0x53, 0x51, 0x4c, 0x5f, 0x55, 0x4e, 0x49, 0x4f, 0x4e, 0x10, 0x00, 0x12, 0x11, + 0x0a, 0x0d, 0x53, 0x51, 0x4c, 0x5f, 0x55, 0x4e, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x4c, 0x4c, 0x10, + 0x01, 0x2a, 0xc9, 0x01, 0x0a, 0x1c, 0x53, 0x71, 0x6c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x73, 0x6f, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x65, 0x76, + 0x65, 0x6c, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, + 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x51, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, - 0x53, 0x45, 0x52, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x04, 0x2a, 0x89, - 0x01, 0x0a, 0x18, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, - 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1f, 0x0a, 0x1b, 0x53, - 0x51, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x55, - 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, - 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x49, 0x54, - 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, - 0x10, 0x01, 0x12, 0x26, 0x0a, 0x22, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x4d, - 0x41, 0x4e, 0x49, 0x50, 0x55, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x52, 0x41, 0x4e, - 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x02, 0x2a, 0xbc, 0x01, 0x0a, 0x19, 0x53, - 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x53, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x51, 0x4c, 0x5f, - 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, - 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x24, 0x0a, - 0x20, 0x53, 0x51, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, - 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x4f, 0x52, 0x57, 0x41, 0x52, 0x44, 0x5f, 0x4f, 0x4e, 0x4c, - 0x59, 0x10, 0x01, 0x12, 0x2a, 0x0a, 0x26, 0x53, 0x51, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, - 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x43, 0x52, 0x4f, 0x4c, - 0x4c, 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x4e, 0x53, 0x49, 0x54, 0x49, 0x56, 0x45, 0x10, 0x02, 0x12, - 0x28, 0x0a, 0x24, 0x53, 0x51, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, - 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x43, 0x52, 0x4f, 0x4c, 0x4c, 0x5f, 0x53, 0x45, - 0x4e, 0x53, 0x49, 0x54, 0x49, 0x56, 0x45, 0x10, 0x03, 0x2a, 0xa2, 0x01, 0x0a, 0x20, 0x53, 0x71, + 0x52, 0x45, 0x41, 0x44, 0x5f, 0x55, 0x4e, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x54, 0x45, 0x44, + 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x53, 0x51, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, + 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x5f, 0x43, 0x4f, 0x4d, 0x4d, 0x49, + 0x54, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x51, 0x4c, 0x5f, 0x54, 0x52, + 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x50, 0x45, 0x41, 0x54, + 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x10, 0x03, 0x12, 0x20, 0x0a, 0x1c, 0x53, + 0x51, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, + 0x45, 0x52, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x04, 0x2a, 0x89, 0x01, + 0x0a, 0x18, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x72, + 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1f, 0x0a, 0x1b, 0x53, 0x51, + 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x4e, + 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x53, + 0x51, 0x4c, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x49, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, + 0x01, 0x12, 0x26, 0x0a, 0x22, 0x53, 0x51, 0x4c, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x4d, 0x41, + 0x4e, 0x49, 0x50, 0x55, 0x4c, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, + 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x02, 0x2a, 0xbc, 0x01, 0x0a, 0x19, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, - 0x53, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x12, 0x2a, - 0x0a, 0x26, 0x53, 0x51, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, - 0x5f, 0x43, 0x4f, 0x4e, 0x43, 0x55, 0x52, 0x52, 0x45, 0x4e, 0x43, 0x59, 0x5f, 0x55, 0x4e, 0x53, - 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x28, 0x0a, 0x24, 0x53, 0x51, - 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x43, 0x4f, 0x4e, - 0x43, 0x55, 0x52, 0x52, 0x45, 0x4e, 0x43, 0x59, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x5f, 0x4f, 0x4e, - 0x4c, 0x59, 0x10, 0x01, 0x12, 0x28, 0x0a, 0x24, 0x53, 0x51, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, - 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x43, 0x4f, 0x4e, 0x43, 0x55, 0x52, 0x52, 0x45, 0x4e, - 0x43, 0x59, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x02, 0x2a, 0x99, - 0x04, 0x0a, 0x12, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x43, 0x6f, - 0x6e, 0x76, 0x65, 0x72, 0x74, 0x12, 0x16, 0x0a, 0x12, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, - 0x56, 0x45, 0x52, 0x54, 0x5f, 0x42, 0x49, 0x47, 0x49, 0x4e, 0x54, 0x10, 0x00, 0x12, 0x16, 0x0a, - 0x12, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x42, 0x49, 0x4e, - 0x41, 0x52, 0x59, 0x10, 0x01, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, - 0x56, 0x45, 0x52, 0x54, 0x5f, 0x42, 0x49, 0x54, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x51, - 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x43, 0x48, 0x41, 0x52, 0x10, 0x03, - 0x12, 0x14, 0x0a, 0x10, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, - 0x44, 0x41, 0x54, 0x45, 0x10, 0x04, 0x12, 0x17, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, - 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x44, 0x45, 0x43, 0x49, 0x4d, 0x41, 0x4c, 0x10, 0x05, 0x12, - 0x15, 0x0a, 0x11, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x46, - 0x4c, 0x4f, 0x41, 0x54, 0x10, 0x06, 0x12, 0x17, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, - 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x47, 0x45, 0x52, 0x10, 0x07, 0x12, - 0x21, 0x0a, 0x1d, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x49, - 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x44, 0x41, 0x59, 0x5f, 0x54, 0x49, 0x4d, 0x45, - 0x10, 0x08, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, - 0x54, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x59, 0x45, 0x41, 0x52, 0x5f, - 0x4d, 0x4f, 0x4e, 0x54, 0x48, 0x10, 0x09, 0x12, 0x1d, 0x0a, 0x19, 0x53, 0x51, 0x4c, 0x5f, 0x43, - 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x4c, 0x4f, 0x4e, 0x47, 0x56, 0x41, 0x52, 0x42, 0x49, - 0x4e, 0x41, 0x52, 0x59, 0x10, 0x0a, 0x12, 0x1b, 0x0a, 0x17, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, - 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x4c, 0x4f, 0x4e, 0x47, 0x56, 0x41, 0x52, 0x43, 0x48, 0x41, - 0x52, 0x10, 0x0b, 0x12, 0x17, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, - 0x52, 0x54, 0x5f, 0x4e, 0x55, 0x4d, 0x45, 0x52, 0x49, 0x43, 0x10, 0x0c, 0x12, 0x14, 0x0a, 0x10, - 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x52, 0x45, 0x41, 0x4c, - 0x10, 0x0d, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, - 0x54, 0x5f, 0x53, 0x4d, 0x41, 0x4c, 0x4c, 0x49, 0x4e, 0x54, 0x10, 0x0e, 0x12, 0x14, 0x0a, 0x10, - 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x54, 0x49, 0x4d, 0x45, - 0x10, 0x0f, 0x12, 0x19, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, - 0x54, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x53, 0x54, 0x41, 0x4d, 0x50, 0x10, 0x10, 0x12, 0x17, 0x0a, - 0x13, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x54, 0x49, 0x4e, - 0x59, 0x49, 0x4e, 0x54, 0x10, 0x11, 0x12, 0x19, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, - 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x56, 0x41, 0x52, 0x42, 0x49, 0x4e, 0x41, 0x52, 0x59, 0x10, - 0x12, 0x12, 0x17, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, - 0x5f, 0x56, 0x41, 0x52, 0x43, 0x48, 0x41, 0x52, 0x10, 0x13, 0x2a, 0x8f, 0x04, 0x0a, 0x0c, 0x58, - 0x64, 0x62, 0x63, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x58, - 0x44, 0x42, 0x43, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, - 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x43, 0x48, 0x41, 0x52, 0x10, - 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x4e, 0x55, 0x4d, 0x45, 0x52, 0x49, - 0x43, 0x10, 0x02, 0x12, 0x10, 0x0a, 0x0c, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x44, 0x45, 0x43, 0x49, - 0x4d, 0x41, 0x4c, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x49, 0x4e, - 0x54, 0x45, 0x47, 0x45, 0x52, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x58, 0x44, 0x42, 0x43, 0x5f, - 0x53, 0x4d, 0x41, 0x4c, 0x4c, 0x49, 0x4e, 0x54, 0x10, 0x05, 0x12, 0x0e, 0x0a, 0x0a, 0x58, 0x44, - 0x42, 0x43, 0x5f, 0x46, 0x4c, 0x4f, 0x41, 0x54, 0x10, 0x06, 0x12, 0x0d, 0x0a, 0x09, 0x58, 0x44, - 0x42, 0x43, 0x5f, 0x52, 0x45, 0x41, 0x4c, 0x10, 0x07, 0x12, 0x0f, 0x0a, 0x0b, 0x58, 0x44, 0x42, - 0x43, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x08, 0x12, 0x11, 0x0a, 0x0d, 0x58, 0x44, - 0x42, 0x43, 0x5f, 0x44, 0x41, 0x54, 0x45, 0x54, 0x49, 0x4d, 0x45, 0x10, 0x09, 0x12, 0x11, 0x0a, - 0x0d, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x10, 0x0a, - 0x12, 0x10, 0x0a, 0x0c, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x56, 0x41, 0x52, 0x43, 0x48, 0x41, 0x52, - 0x10, 0x0c, 0x12, 0x0d, 0x0a, 0x09, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x44, 0x41, 0x54, 0x45, 0x10, - 0x5b, 0x12, 0x0d, 0x0a, 0x09, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x10, 0x5c, - 0x12, 0x12, 0x0a, 0x0e, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x53, 0x54, 0x41, - 0x4d, 0x50, 0x10, 0x5d, 0x12, 0x1d, 0x0a, 0x10, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x4c, 0x4f, 0x4e, - 0x47, 0x56, 0x41, 0x52, 0x43, 0x48, 0x41, 0x52, 0x10, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, - 0xff, 0xff, 0x01, 0x12, 0x18, 0x0a, 0x0b, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x42, 0x49, 0x4e, 0x41, - 0x52, 0x59, 0x10, 0xfe, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x1b, 0x0a, - 0x0e, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x56, 0x41, 0x52, 0x42, 0x49, 0x4e, 0x41, 0x52, 0x59, 0x10, - 0xfd, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x1f, 0x0a, 0x12, 0x58, 0x44, - 0x42, 0x43, 0x5f, 0x4c, 0x4f, 0x4e, 0x47, 0x56, 0x41, 0x52, 0x42, 0x49, 0x4e, 0x41, 0x52, 0x59, - 0x10, 0xfc, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x18, 0x0a, 0x0b, 0x58, - 0x44, 0x42, 0x43, 0x5f, 0x42, 0x49, 0x47, 0x49, 0x4e, 0x54, 0x10, 0xfb, 0xff, 0xff, 0xff, 0xff, - 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x19, 0x0a, 0x0c, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x54, 0x49, - 0x4e, 0x59, 0x49, 0x4e, 0x54, 0x10, 0xfa, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, - 0x12, 0x15, 0x0a, 0x08, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x42, 0x49, 0x54, 0x10, 0xf9, 0xff, 0xff, - 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x17, 0x0a, 0x0a, 0x58, 0x44, 0x42, 0x43, 0x5f, - 0x57, 0x43, 0x48, 0x41, 0x52, 0x10, 0xf8, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, - 0x12, 0x1a, 0x0a, 0x0d, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x57, 0x56, 0x41, 0x52, 0x43, 0x48, 0x41, - 0x52, 0x10, 0xf7, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x2a, 0xa3, 0x08, 0x0a, - 0x13, 0x58, 0x64, 0x62, 0x63, 0x44, 0x61, 0x74, 0x65, 0x74, 0x69, 0x6d, 0x65, 0x53, 0x75, 0x62, - 0x63, 0x6f, 0x64, 0x65, 0x12, 0x18, 0x0a, 0x14, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, - 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x15, - 0x0a, 0x11, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x59, - 0x45, 0x41, 0x52, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, - 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x44, 0x41, 0x54, 0x45, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, - 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x54, 0x49, 0x4d, - 0x45, 0x10, 0x02, 0x12, 0x16, 0x0a, 0x12, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, - 0x4f, 0x44, 0x45, 0x5f, 0x4d, 0x4f, 0x4e, 0x54, 0x48, 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x58, + 0x53, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x51, 0x4c, 0x5f, 0x52, + 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, + 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, + 0x53, 0x51, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x54, + 0x59, 0x50, 0x45, 0x5f, 0x46, 0x4f, 0x52, 0x57, 0x41, 0x52, 0x44, 0x5f, 0x4f, 0x4e, 0x4c, 0x59, + 0x10, 0x01, 0x12, 0x2a, 0x0a, 0x26, 0x53, 0x51, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, + 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x43, 0x52, 0x4f, 0x4c, 0x4c, + 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x4e, 0x53, 0x49, 0x54, 0x49, 0x56, 0x45, 0x10, 0x02, 0x12, 0x28, + 0x0a, 0x24, 0x53, 0x51, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, + 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x43, 0x52, 0x4f, 0x4c, 0x4c, 0x5f, 0x53, 0x45, 0x4e, + 0x53, 0x49, 0x54, 0x49, 0x56, 0x45, 0x10, 0x03, 0x2a, 0xa2, 0x01, 0x0a, 0x20, 0x53, 0x71, 0x6c, + 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x53, + 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x12, 0x2a, 0x0a, + 0x26, 0x53, 0x51, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, + 0x43, 0x4f, 0x4e, 0x43, 0x55, 0x52, 0x52, 0x45, 0x4e, 0x43, 0x59, 0x5f, 0x55, 0x4e, 0x53, 0x50, + 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x28, 0x0a, 0x24, 0x53, 0x51, 0x4c, + 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x43, 0x4f, 0x4e, 0x43, + 0x55, 0x52, 0x52, 0x45, 0x4e, 0x43, 0x59, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x5f, 0x4f, 0x4e, 0x4c, + 0x59, 0x10, 0x01, 0x12, 0x28, 0x0a, 0x24, 0x53, 0x51, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, + 0x54, 0x5f, 0x53, 0x45, 0x54, 0x5f, 0x43, 0x4f, 0x4e, 0x43, 0x55, 0x52, 0x52, 0x45, 0x4e, 0x43, + 0x59, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x02, 0x2a, 0x99, 0x04, + 0x0a, 0x12, 0x53, 0x71, 0x6c, 0x53, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x43, 0x6f, 0x6e, + 0x76, 0x65, 0x72, 0x74, 0x12, 0x16, 0x0a, 0x12, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, + 0x45, 0x52, 0x54, 0x5f, 0x42, 0x49, 0x47, 0x49, 0x4e, 0x54, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, + 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x42, 0x49, 0x4e, 0x41, + 0x52, 0x59, 0x10, 0x01, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, + 0x45, 0x52, 0x54, 0x5f, 0x42, 0x49, 0x54, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x51, 0x4c, + 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x43, 0x48, 0x41, 0x52, 0x10, 0x03, 0x12, + 0x14, 0x0a, 0x10, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x44, + 0x41, 0x54, 0x45, 0x10, 0x04, 0x12, 0x17, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, + 0x56, 0x45, 0x52, 0x54, 0x5f, 0x44, 0x45, 0x43, 0x49, 0x4d, 0x41, 0x4c, 0x10, 0x05, 0x12, 0x15, + 0x0a, 0x11, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x46, 0x4c, + 0x4f, 0x41, 0x54, 0x10, 0x06, 0x12, 0x17, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, + 0x56, 0x45, 0x52, 0x54, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x47, 0x45, 0x52, 0x10, 0x07, 0x12, 0x21, + 0x0a, 0x1d, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x49, 0x4e, + 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x44, 0x41, 0x59, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x10, + 0x08, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, + 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x59, 0x45, 0x41, 0x52, 0x5f, 0x4d, + 0x4f, 0x4e, 0x54, 0x48, 0x10, 0x09, 0x12, 0x1d, 0x0a, 0x19, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, + 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x4c, 0x4f, 0x4e, 0x47, 0x56, 0x41, 0x52, 0x42, 0x49, 0x4e, + 0x41, 0x52, 0x59, 0x10, 0x0a, 0x12, 0x1b, 0x0a, 0x17, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, + 0x56, 0x45, 0x52, 0x54, 0x5f, 0x4c, 0x4f, 0x4e, 0x47, 0x56, 0x41, 0x52, 0x43, 0x48, 0x41, 0x52, + 0x10, 0x0b, 0x12, 0x17, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, + 0x54, 0x5f, 0x4e, 0x55, 0x4d, 0x45, 0x52, 0x49, 0x43, 0x10, 0x0c, 0x12, 0x14, 0x0a, 0x10, 0x53, + 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x52, 0x45, 0x41, 0x4c, 0x10, + 0x0d, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, + 0x5f, 0x53, 0x4d, 0x41, 0x4c, 0x4c, 0x49, 0x4e, 0x54, 0x10, 0x0e, 0x12, 0x14, 0x0a, 0x10, 0x53, + 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x10, + 0x0f, 0x12, 0x19, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, + 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x53, 0x54, 0x41, 0x4d, 0x50, 0x10, 0x10, 0x12, 0x17, 0x0a, 0x13, + 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, 0x54, 0x49, 0x4e, 0x59, + 0x49, 0x4e, 0x54, 0x10, 0x11, 0x12, 0x19, 0x0a, 0x15, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, + 0x56, 0x45, 0x52, 0x54, 0x5f, 0x56, 0x41, 0x52, 0x42, 0x49, 0x4e, 0x41, 0x52, 0x59, 0x10, 0x12, + 0x12, 0x17, 0x0a, 0x13, 0x53, 0x51, 0x4c, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x5f, + 0x56, 0x41, 0x52, 0x43, 0x48, 0x41, 0x52, 0x10, 0x13, 0x2a, 0x8f, 0x04, 0x0a, 0x0c, 0x58, 0x64, + 0x62, 0x63, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x58, 0x44, + 0x42, 0x43, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, + 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x43, 0x48, 0x41, 0x52, 0x10, 0x01, + 0x12, 0x10, 0x0a, 0x0c, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x4e, 0x55, 0x4d, 0x45, 0x52, 0x49, 0x43, + 0x10, 0x02, 0x12, 0x10, 0x0a, 0x0c, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x44, 0x45, 0x43, 0x49, 0x4d, + 0x41, 0x4c, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x49, 0x4e, 0x54, + 0x45, 0x47, 0x45, 0x52, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, + 0x4d, 0x41, 0x4c, 0x4c, 0x49, 0x4e, 0x54, 0x10, 0x05, 0x12, 0x0e, 0x0a, 0x0a, 0x58, 0x44, 0x42, + 0x43, 0x5f, 0x46, 0x4c, 0x4f, 0x41, 0x54, 0x10, 0x06, 0x12, 0x0d, 0x0a, 0x09, 0x58, 0x44, 0x42, + 0x43, 0x5f, 0x52, 0x45, 0x41, 0x4c, 0x10, 0x07, 0x12, 0x0f, 0x0a, 0x0b, 0x58, 0x44, 0x42, 0x43, + 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x08, 0x12, 0x11, 0x0a, 0x0d, 0x58, 0x44, 0x42, + 0x43, 0x5f, 0x44, 0x41, 0x54, 0x45, 0x54, 0x49, 0x4d, 0x45, 0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, + 0x58, 0x44, 0x42, 0x43, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x10, 0x0a, 0x12, + 0x10, 0x0a, 0x0c, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x56, 0x41, 0x52, 0x43, 0x48, 0x41, 0x52, 0x10, + 0x0c, 0x12, 0x0d, 0x0a, 0x09, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x44, 0x41, 0x54, 0x45, 0x10, 0x5b, + 0x12, 0x0d, 0x0a, 0x09, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x10, 0x5c, 0x12, + 0x12, 0x0a, 0x0e, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x53, 0x54, 0x41, 0x4d, + 0x50, 0x10, 0x5d, 0x12, 0x1d, 0x0a, 0x10, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x4c, 0x4f, 0x4e, 0x47, + 0x56, 0x41, 0x52, 0x43, 0x48, 0x41, 0x52, 0x10, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, + 0xff, 0x01, 0x12, 0x18, 0x0a, 0x0b, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x42, 0x49, 0x4e, 0x41, 0x52, + 0x59, 0x10, 0xfe, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x1b, 0x0a, 0x0e, + 0x58, 0x44, 0x42, 0x43, 0x5f, 0x56, 0x41, 0x52, 0x42, 0x49, 0x4e, 0x41, 0x52, 0x59, 0x10, 0xfd, + 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x1f, 0x0a, 0x12, 0x58, 0x44, 0x42, + 0x43, 0x5f, 0x4c, 0x4f, 0x4e, 0x47, 0x56, 0x41, 0x52, 0x42, 0x49, 0x4e, 0x41, 0x52, 0x59, 0x10, + 0xfc, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x18, 0x0a, 0x0b, 0x58, 0x44, + 0x42, 0x43, 0x5f, 0x42, 0x49, 0x47, 0x49, 0x4e, 0x54, 0x10, 0xfb, 0xff, 0xff, 0xff, 0xff, 0xff, + 0xff, 0xff, 0xff, 0x01, 0x12, 0x19, 0x0a, 0x0c, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x54, 0x49, 0x4e, + 0x59, 0x49, 0x4e, 0x54, 0x10, 0xfa, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, + 0x15, 0x0a, 0x08, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x42, 0x49, 0x54, 0x10, 0xf9, 0xff, 0xff, 0xff, + 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x17, 0x0a, 0x0a, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x57, + 0x43, 0x48, 0x41, 0x52, 0x10, 0xf8, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, + 0x1a, 0x0a, 0x0d, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x57, 0x56, 0x41, 0x52, 0x43, 0x48, 0x41, 0x52, + 0x10, 0xf7, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x2a, 0xa3, 0x08, 0x0a, 0x13, + 0x58, 0x64, 0x62, 0x63, 0x44, 0x61, 0x74, 0x65, 0x74, 0x69, 0x6d, 0x65, 0x53, 0x75, 0x62, 0x63, + 0x6f, 0x64, 0x65, 0x12, 0x18, 0x0a, 0x14, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, + 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x15, 0x0a, + 0x11, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x59, 0x45, + 0x41, 0x52, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, + 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x44, 0x41, 0x54, 0x45, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x54, 0x49, 0x4d, 0x45, - 0x53, 0x54, 0x41, 0x4d, 0x50, 0x10, 0x03, 0x12, 0x14, 0x0a, 0x10, 0x58, 0x44, 0x42, 0x43, 0x5f, - 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x44, 0x41, 0x59, 0x10, 0x03, 0x12, 0x23, 0x0a, - 0x1f, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x54, 0x49, - 0x4d, 0x45, 0x5f, 0x57, 0x49, 0x54, 0x48, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x5a, 0x4f, 0x4e, 0x45, - 0x10, 0x04, 0x12, 0x15, 0x0a, 0x11, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, - 0x44, 0x45, 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x10, 0x04, 0x12, 0x28, 0x0a, 0x24, 0x58, 0x44, 0x42, - 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x53, 0x54, - 0x41, 0x4d, 0x50, 0x5f, 0x57, 0x49, 0x54, 0x48, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x5a, 0x4f, 0x4e, - 0x45, 0x10, 0x05, 0x12, 0x17, 0x0a, 0x13, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, - 0x4f, 0x44, 0x45, 0x5f, 0x4d, 0x49, 0x4e, 0x55, 0x54, 0x45, 0x10, 0x05, 0x12, 0x17, 0x0a, 0x13, - 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x53, 0x45, 0x43, - 0x4f, 0x4e, 0x44, 0x10, 0x06, 0x12, 0x1e, 0x0a, 0x1a, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, - 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x59, 0x45, 0x41, 0x52, 0x5f, 0x54, 0x4f, 0x5f, 0x4d, 0x4f, - 0x4e, 0x54, 0x48, 0x10, 0x07, 0x12, 0x1c, 0x0a, 0x18, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, - 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x44, 0x41, 0x59, 0x5f, 0x54, 0x4f, 0x5f, 0x48, 0x4f, 0x55, - 0x52, 0x10, 0x08, 0x12, 0x1e, 0x0a, 0x1a, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, - 0x4f, 0x44, 0x45, 0x5f, 0x44, 0x41, 0x59, 0x5f, 0x54, 0x4f, 0x5f, 0x4d, 0x49, 0x4e, 0x55, 0x54, - 0x45, 0x10, 0x09, 0x12, 0x1e, 0x0a, 0x1a, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, - 0x4f, 0x44, 0x45, 0x5f, 0x44, 0x41, 0x59, 0x5f, 0x54, 0x4f, 0x5f, 0x53, 0x45, 0x43, 0x4f, 0x4e, - 0x44, 0x10, 0x0a, 0x12, 0x1f, 0x0a, 0x1b, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, - 0x4f, 0x44, 0x45, 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x5f, 0x54, 0x4f, 0x5f, 0x4d, 0x49, 0x4e, 0x55, - 0x54, 0x45, 0x10, 0x0b, 0x12, 0x1f, 0x0a, 0x1b, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, - 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x5f, 0x54, 0x4f, 0x5f, 0x53, 0x45, 0x43, - 0x4f, 0x4e, 0x44, 0x10, 0x0c, 0x12, 0x21, 0x0a, 0x1d, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, - 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4d, 0x49, 0x4e, 0x55, 0x54, 0x45, 0x5f, 0x54, 0x4f, 0x5f, - 0x53, 0x45, 0x43, 0x4f, 0x4e, 0x44, 0x10, 0x0d, 0x12, 0x1e, 0x0a, 0x1a, 0x58, 0x44, 0x42, 0x43, - 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, - 0x4c, 0x5f, 0x59, 0x45, 0x41, 0x52, 0x10, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x58, 0x44, 0x42, 0x43, + 0x10, 0x02, 0x12, 0x16, 0x0a, 0x12, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, + 0x44, 0x45, 0x5f, 0x4d, 0x4f, 0x4e, 0x54, 0x48, 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x58, 0x44, + 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x53, + 0x54, 0x41, 0x4d, 0x50, 0x10, 0x03, 0x12, 0x14, 0x0a, 0x10, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, + 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x44, 0x41, 0x59, 0x10, 0x03, 0x12, 0x23, 0x0a, 0x1f, + 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x54, 0x49, 0x4d, + 0x45, 0x5f, 0x57, 0x49, 0x54, 0x48, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x5a, 0x4f, 0x4e, 0x45, 0x10, + 0x04, 0x12, 0x15, 0x0a, 0x11, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, + 0x45, 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x10, 0x04, 0x12, 0x28, 0x0a, 0x24, 0x58, 0x44, 0x42, 0x43, + 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x53, 0x54, 0x41, + 0x4d, 0x50, 0x5f, 0x57, 0x49, 0x54, 0x48, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x5a, 0x4f, 0x4e, 0x45, + 0x10, 0x05, 0x12, 0x17, 0x0a, 0x13, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, + 0x44, 0x45, 0x5f, 0x4d, 0x49, 0x4e, 0x55, 0x54, 0x45, 0x10, 0x05, 0x12, 0x17, 0x0a, 0x13, 0x58, + 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x53, 0x45, 0x43, 0x4f, + 0x4e, 0x44, 0x10, 0x06, 0x12, 0x1e, 0x0a, 0x1a, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, + 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x59, 0x45, 0x41, 0x52, 0x5f, 0x54, 0x4f, 0x5f, 0x4d, 0x4f, 0x4e, + 0x54, 0x48, 0x10, 0x07, 0x12, 0x1c, 0x0a, 0x18, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, + 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x44, 0x41, 0x59, 0x5f, 0x54, 0x4f, 0x5f, 0x48, 0x4f, 0x55, 0x52, + 0x10, 0x08, 0x12, 0x1e, 0x0a, 0x1a, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, + 0x44, 0x45, 0x5f, 0x44, 0x41, 0x59, 0x5f, 0x54, 0x4f, 0x5f, 0x4d, 0x49, 0x4e, 0x55, 0x54, 0x45, + 0x10, 0x09, 0x12, 0x1e, 0x0a, 0x1a, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, + 0x44, 0x45, 0x5f, 0x44, 0x41, 0x59, 0x5f, 0x54, 0x4f, 0x5f, 0x53, 0x45, 0x43, 0x4f, 0x4e, 0x44, + 0x10, 0x0a, 0x12, 0x1f, 0x0a, 0x1b, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, + 0x44, 0x45, 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x5f, 0x54, 0x4f, 0x5f, 0x4d, 0x49, 0x4e, 0x55, 0x54, + 0x45, 0x10, 0x0b, 0x12, 0x1f, 0x0a, 0x1b, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, + 0x4f, 0x44, 0x45, 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x5f, 0x54, 0x4f, 0x5f, 0x53, 0x45, 0x43, 0x4f, + 0x4e, 0x44, 0x10, 0x0c, 0x12, 0x21, 0x0a, 0x1d, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, + 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4d, 0x49, 0x4e, 0x55, 0x54, 0x45, 0x5f, 0x54, 0x4f, 0x5f, 0x53, + 0x45, 0x43, 0x4f, 0x4e, 0x44, 0x10, 0x0d, 0x12, 0x1e, 0x0a, 0x1a, 0x58, 0x44, 0x42, 0x43, 0x5f, + 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, + 0x5f, 0x59, 0x45, 0x41, 0x52, 0x10, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x58, 0x44, 0x42, 0x43, 0x5f, + 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, + 0x5f, 0x4d, 0x4f, 0x4e, 0x54, 0x48, 0x10, 0x66, 0x12, 0x1d, 0x0a, 0x19, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, - 0x4c, 0x5f, 0x4d, 0x4f, 0x4e, 0x54, 0x48, 0x10, 0x66, 0x12, 0x1d, 0x0a, 0x19, 0x58, 0x44, 0x42, + 0x4c, 0x5f, 0x44, 0x41, 0x59, 0x10, 0x67, 0x12, 0x1e, 0x0a, 0x1a, 0x58, 0x44, 0x42, 0x43, 0x5f, + 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, + 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x10, 0x68, 0x12, 0x20, 0x0a, 0x1c, 0x58, 0x44, 0x42, 0x43, 0x5f, + 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, + 0x5f, 0x4d, 0x49, 0x4e, 0x55, 0x54, 0x45, 0x10, 0x69, 0x12, 0x20, 0x0a, 0x1c, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, - 0x41, 0x4c, 0x5f, 0x44, 0x41, 0x59, 0x10, 0x67, 0x12, 0x1e, 0x0a, 0x1a, 0x58, 0x44, 0x42, 0x43, - 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, - 0x4c, 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x10, 0x68, 0x12, 0x20, 0x0a, 0x1c, 0x58, 0x44, 0x42, 0x43, - 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, - 0x4c, 0x5f, 0x4d, 0x49, 0x4e, 0x55, 0x54, 0x45, 0x10, 0x69, 0x12, 0x20, 0x0a, 0x1c, 0x58, 0x44, - 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, - 0x56, 0x41, 0x4c, 0x5f, 0x53, 0x45, 0x43, 0x4f, 0x4e, 0x44, 0x10, 0x6a, 0x12, 0x27, 0x0a, 0x23, - 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, - 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x59, 0x45, 0x41, 0x52, 0x5f, 0x54, 0x4f, 0x5f, 0x4d, 0x4f, - 0x4e, 0x54, 0x48, 0x10, 0x6b, 0x12, 0x25, 0x0a, 0x21, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, - 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x44, - 0x41, 0x59, 0x5f, 0x54, 0x4f, 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x10, 0x6c, 0x12, 0x27, 0x0a, 0x23, - 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, - 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x44, 0x41, 0x59, 0x5f, 0x54, 0x4f, 0x5f, 0x4d, 0x49, 0x4e, - 0x55, 0x54, 0x45, 0x10, 0x6d, 0x12, 0x27, 0x0a, 0x23, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, - 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x44, - 0x41, 0x59, 0x5f, 0x54, 0x4f, 0x5f, 0x53, 0x45, 0x43, 0x4f, 0x4e, 0x44, 0x10, 0x6e, 0x12, 0x28, - 0x0a, 0x24, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, - 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x5f, 0x54, 0x4f, 0x5f, - 0x4d, 0x49, 0x4e, 0x55, 0x54, 0x45, 0x10, 0x6f, 0x12, 0x28, 0x0a, 0x24, 0x58, 0x44, 0x42, 0x43, - 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, - 0x4c, 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x5f, 0x54, 0x4f, 0x5f, 0x53, 0x45, 0x43, 0x4f, 0x4e, 0x44, - 0x10, 0x70, 0x12, 0x2a, 0x0a, 0x26, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, - 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x4d, 0x49, 0x4e, 0x55, - 0x54, 0x45, 0x5f, 0x54, 0x4f, 0x5f, 0x53, 0x45, 0x43, 0x4f, 0x4e, 0x44, 0x10, 0x71, 0x1a, 0x02, - 0x10, 0x01, 0x2a, 0x57, 0x0a, 0x08, 0x4e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x18, - 0x0a, 0x14, 0x4e, 0x55, 0x4c, 0x4c, 0x41, 0x42, 0x49, 0x4c, 0x49, 0x54, 0x59, 0x5f, 0x4e, 0x4f, - 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x53, 0x10, 0x00, 0x12, 0x18, 0x0a, 0x14, 0x4e, 0x55, 0x4c, 0x4c, - 0x41, 0x42, 0x49, 0x4c, 0x49, 0x54, 0x59, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x41, 0x42, 0x4c, 0x45, - 0x10, 0x01, 0x12, 0x17, 0x0a, 0x13, 0x4e, 0x55, 0x4c, 0x4c, 0x41, 0x42, 0x49, 0x4c, 0x49, 0x54, - 0x59, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x02, 0x2a, 0x61, 0x0a, 0x0a, 0x53, - 0x65, 0x61, 0x72, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x45, 0x41, - 0x52, 0x43, 0x48, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x13, - 0x0a, 0x0f, 0x53, 0x45, 0x41, 0x52, 0x43, 0x48, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x43, 0x48, 0x41, - 0x52, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x45, 0x41, 0x52, 0x43, 0x48, 0x41, 0x42, 0x4c, - 0x45, 0x5f, 0x42, 0x41, 0x53, 0x49, 0x43, 0x10, 0x02, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x45, 0x41, - 0x52, 0x43, 0x48, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x46, 0x55, 0x4c, 0x4c, 0x10, 0x03, 0x2a, 0x5c, - 0x0a, 0x11, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x75, - 0x6c, 0x65, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x41, 0x53, 0x43, 0x41, 0x44, 0x45, 0x10, 0x00, - 0x12, 0x0c, 0x0a, 0x08, 0x52, 0x45, 0x53, 0x54, 0x52, 0x49, 0x43, 0x54, 0x10, 0x01, 0x12, 0x0c, - 0x0a, 0x08, 0x53, 0x45, 0x54, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, - 0x4e, 0x4f, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x03, 0x12, 0x0f, 0x0a, 0x0b, 0x53, - 0x45, 0x54, 0x5f, 0x44, 0x45, 0x46, 0x41, 0x55, 0x4c, 0x54, 0x10, 0x04, 0x3a, 0x44, 0x0a, 0x0c, - 0x65, 0x78, 0x70, 0x65, 0x72, 0x69, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x12, 0x1f, 0x2e, 0x67, - 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xe8, 0x07, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x65, 0x78, 0x70, 0x65, 0x72, 0x69, 0x6d, 0x65, 0x6e, 0x74, - 0x61, 0x6c, 0x42, 0x56, 0x0a, 0x20, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, - 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x73, 0x71, - 0x6c, 0x2e, 0x69, 0x6d, 0x70, 0x6c, 0x5a, 0x32, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, - 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2f, - 0x67, 0x6f, 0x2f, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2f, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2f, - 0x67, 0x65, 0x6e, 0x2f, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x33, + 0x41, 0x4c, 0x5f, 0x53, 0x45, 0x43, 0x4f, 0x4e, 0x44, 0x10, 0x6a, 0x12, 0x27, 0x0a, 0x23, 0x58, + 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, + 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x59, 0x45, 0x41, 0x52, 0x5f, 0x54, 0x4f, 0x5f, 0x4d, 0x4f, 0x4e, + 0x54, 0x48, 0x10, 0x6b, 0x12, 0x25, 0x0a, 0x21, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, + 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x44, 0x41, + 0x59, 0x5f, 0x54, 0x4f, 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x10, 0x6c, 0x12, 0x27, 0x0a, 0x23, 0x58, + 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, + 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x44, 0x41, 0x59, 0x5f, 0x54, 0x4f, 0x5f, 0x4d, 0x49, 0x4e, 0x55, + 0x54, 0x45, 0x10, 0x6d, 0x12, 0x27, 0x0a, 0x23, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, + 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x44, 0x41, + 0x59, 0x5f, 0x54, 0x4f, 0x5f, 0x53, 0x45, 0x43, 0x4f, 0x4e, 0x44, 0x10, 0x6e, 0x12, 0x28, 0x0a, + 0x24, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, + 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x5f, 0x54, 0x4f, 0x5f, 0x4d, + 0x49, 0x4e, 0x55, 0x54, 0x45, 0x10, 0x6f, 0x12, 0x28, 0x0a, 0x24, 0x58, 0x44, 0x42, 0x43, 0x5f, + 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, + 0x5f, 0x48, 0x4f, 0x55, 0x52, 0x5f, 0x54, 0x4f, 0x5f, 0x53, 0x45, 0x43, 0x4f, 0x4e, 0x44, 0x10, + 0x70, 0x12, 0x2a, 0x0a, 0x26, 0x58, 0x44, 0x42, 0x43, 0x5f, 0x53, 0x55, 0x42, 0x43, 0x4f, 0x44, + 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x4d, 0x49, 0x4e, 0x55, 0x54, + 0x45, 0x5f, 0x54, 0x4f, 0x5f, 0x53, 0x45, 0x43, 0x4f, 0x4e, 0x44, 0x10, 0x71, 0x1a, 0x02, 0x10, + 0x01, 0x2a, 0x57, 0x0a, 0x08, 0x4e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x18, 0x0a, + 0x14, 0x4e, 0x55, 0x4c, 0x4c, 0x41, 0x42, 0x49, 0x4c, 0x49, 0x54, 0x59, 0x5f, 0x4e, 0x4f, 0x5f, + 0x4e, 0x55, 0x4c, 0x4c, 0x53, 0x10, 0x00, 0x12, 0x18, 0x0a, 0x14, 0x4e, 0x55, 0x4c, 0x4c, 0x41, + 0x42, 0x49, 0x4c, 0x49, 0x54, 0x59, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, + 0x01, 0x12, 0x17, 0x0a, 0x13, 0x4e, 0x55, 0x4c, 0x4c, 0x41, 0x42, 0x49, 0x4c, 0x49, 0x54, 0x59, + 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x02, 0x2a, 0x61, 0x0a, 0x0a, 0x53, 0x65, + 0x61, 0x72, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x45, 0x41, 0x52, + 0x43, 0x48, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x13, 0x0a, + 0x0f, 0x53, 0x45, 0x41, 0x52, 0x43, 0x48, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x43, 0x48, 0x41, 0x52, + 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x45, 0x41, 0x52, 0x43, 0x48, 0x41, 0x42, 0x4c, 0x45, + 0x5f, 0x42, 0x41, 0x53, 0x49, 0x43, 0x10, 0x02, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x45, 0x41, 0x52, + 0x43, 0x48, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x46, 0x55, 0x4c, 0x4c, 0x10, 0x03, 0x2a, 0x5c, 0x0a, + 0x11, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x75, 0x6c, + 0x65, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x41, 0x53, 0x43, 0x41, 0x44, 0x45, 0x10, 0x00, 0x12, + 0x0c, 0x0a, 0x08, 0x52, 0x45, 0x53, 0x54, 0x52, 0x49, 0x43, 0x54, 0x10, 0x01, 0x12, 0x0c, 0x0a, + 0x08, 0x53, 0x45, 0x54, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x4e, + 0x4f, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x03, 0x12, 0x0f, 0x0a, 0x0b, 0x53, 0x45, + 0x54, 0x5f, 0x44, 0x45, 0x46, 0x41, 0x55, 0x4c, 0x54, 0x10, 0x04, 0x3a, 0x44, 0x0a, 0x0c, 0x65, + 0x78, 0x70, 0x65, 0x72, 0x69, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x12, 0x1f, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xe8, 0x07, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0c, 0x65, 0x78, 0x70, 0x65, 0x72, 0x69, 0x6d, 0x65, 0x6e, 0x74, 0x61, + 0x6c, 0x42, 0x56, 0x0a, 0x20, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x73, 0x71, 0x6c, + 0x2e, 0x69, 0x6d, 0x70, 0x6c, 0x5a, 0x32, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, + 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2f, 0x67, + 0x6f, 0x2f, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2f, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2f, 0x67, + 0x65, 0x6e, 0x2f, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, } var ( @@ -5127,78 +5299,87 @@ func file_FlightSql_proto_rawDescGZIP() []byte { return file_FlightSql_proto_rawDescData } -var file_FlightSql_proto_enumTypes = make([]protoimpl.EnumInfo, 25) -var file_FlightSql_proto_msgTypes = make([]protoimpl.MessageInfo, 30) +var file_FlightSql_proto_enumTypes = make([]protoimpl.EnumInfo, 27) +var file_FlightSql_proto_msgTypes = make([]protoimpl.MessageInfo, 33) var file_FlightSql_proto_goTypes = []interface{}{ - (SqlInfo)(0), // 0: arrow.flight.protocol.sql.SqlInfo - (SqlSupportedTransaction)(0), // 1: arrow.flight.protocol.sql.SqlSupportedTransaction - (SqlSupportedCaseSensitivity)(0), // 2: arrow.flight.protocol.sql.SqlSupportedCaseSensitivity - (SqlNullOrdering)(0), // 3: arrow.flight.protocol.sql.SqlNullOrdering - (SupportedSqlGrammar)(0), // 4: arrow.flight.protocol.sql.SupportedSqlGrammar - (SupportedAnsi92SqlGrammarLevel)(0), // 5: arrow.flight.protocol.sql.SupportedAnsi92SqlGrammarLevel - (SqlOuterJoinsSupportLevel)(0), // 6: arrow.flight.protocol.sql.SqlOuterJoinsSupportLevel - (SqlSupportedGroupBy)(0), // 7: arrow.flight.protocol.sql.SqlSupportedGroupBy - (SqlSupportedElementActions)(0), // 8: arrow.flight.protocol.sql.SqlSupportedElementActions - (SqlSupportedPositionedCommands)(0), // 9: arrow.flight.protocol.sql.SqlSupportedPositionedCommands - (SqlSupportedSubqueries)(0), // 10: arrow.flight.protocol.sql.SqlSupportedSubqueries - (SqlSupportedUnions)(0), // 11: arrow.flight.protocol.sql.SqlSupportedUnions - (SqlTransactionIsolationLevel)(0), // 12: arrow.flight.protocol.sql.SqlTransactionIsolationLevel - (SqlSupportedTransactions)(0), // 13: arrow.flight.protocol.sql.SqlSupportedTransactions - (SqlSupportedResultSetType)(0), // 14: arrow.flight.protocol.sql.SqlSupportedResultSetType - (SqlSupportedResultSetConcurrency)(0), // 15: arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency - (SqlSupportsConvert)(0), // 16: arrow.flight.protocol.sql.SqlSupportsConvert - (XdbcDataType)(0), // 17: arrow.flight.protocol.sql.XdbcDataType - (XdbcDatetimeSubcode)(0), // 18: arrow.flight.protocol.sql.XdbcDatetimeSubcode - (Nullable)(0), // 19: arrow.flight.protocol.sql.Nullable - (Searchable)(0), // 20: arrow.flight.protocol.sql.Searchable - (UpdateDeleteRules)(0), // 21: arrow.flight.protocol.sql.UpdateDeleteRules - (ActionEndTransactionRequest_EndTransaction)(0), // 22: arrow.flight.protocol.sql.ActionEndTransactionRequest.EndTransaction - (ActionEndSavepointRequest_EndSavepoint)(0), // 23: arrow.flight.protocol.sql.ActionEndSavepointRequest.EndSavepoint - (ActionCancelQueryResult_CancelResult)(0), // 24: arrow.flight.protocol.sql.ActionCancelQueryResult.CancelResult - (*CommandGetSqlInfo)(nil), // 25: arrow.flight.protocol.sql.CommandGetSqlInfo - (*CommandGetXdbcTypeInfo)(nil), // 26: arrow.flight.protocol.sql.CommandGetXdbcTypeInfo - (*CommandGetCatalogs)(nil), // 27: arrow.flight.protocol.sql.CommandGetCatalogs - (*CommandGetDbSchemas)(nil), // 28: arrow.flight.protocol.sql.CommandGetDbSchemas - (*CommandGetTables)(nil), // 29: arrow.flight.protocol.sql.CommandGetTables - (*CommandGetTableTypes)(nil), // 30: arrow.flight.protocol.sql.CommandGetTableTypes - (*CommandGetPrimaryKeys)(nil), // 31: arrow.flight.protocol.sql.CommandGetPrimaryKeys - (*CommandGetExportedKeys)(nil), // 32: arrow.flight.protocol.sql.CommandGetExportedKeys - (*CommandGetImportedKeys)(nil), // 33: arrow.flight.protocol.sql.CommandGetImportedKeys - (*CommandGetCrossReference)(nil), // 34: arrow.flight.protocol.sql.CommandGetCrossReference - (*ActionCreatePreparedStatementRequest)(nil), // 35: arrow.flight.protocol.sql.ActionCreatePreparedStatementRequest - (*SubstraitPlan)(nil), // 36: arrow.flight.protocol.sql.SubstraitPlan - (*ActionCreatePreparedSubstraitPlanRequest)(nil), // 37: arrow.flight.protocol.sql.ActionCreatePreparedSubstraitPlanRequest - (*ActionCreatePreparedStatementResult)(nil), // 38: arrow.flight.protocol.sql.ActionCreatePreparedStatementResult - (*ActionClosePreparedStatementRequest)(nil), // 39: arrow.flight.protocol.sql.ActionClosePreparedStatementRequest - (*ActionBeginTransactionRequest)(nil), // 40: arrow.flight.protocol.sql.ActionBeginTransactionRequest - (*ActionBeginSavepointRequest)(nil), // 41: arrow.flight.protocol.sql.ActionBeginSavepointRequest - (*ActionBeginTransactionResult)(nil), // 42: arrow.flight.protocol.sql.ActionBeginTransactionResult - (*ActionBeginSavepointResult)(nil), // 43: arrow.flight.protocol.sql.ActionBeginSavepointResult - (*ActionEndTransactionRequest)(nil), // 44: arrow.flight.protocol.sql.ActionEndTransactionRequest - (*ActionEndSavepointRequest)(nil), // 45: arrow.flight.protocol.sql.ActionEndSavepointRequest - (*CommandStatementQuery)(nil), // 46: arrow.flight.protocol.sql.CommandStatementQuery - (*CommandStatementSubstraitPlan)(nil), // 47: arrow.flight.protocol.sql.CommandStatementSubstraitPlan - (*TicketStatementQuery)(nil), // 48: arrow.flight.protocol.sql.TicketStatementQuery - (*CommandPreparedStatementQuery)(nil), // 49: arrow.flight.protocol.sql.CommandPreparedStatementQuery - (*CommandStatementUpdate)(nil), // 50: arrow.flight.protocol.sql.CommandStatementUpdate - (*CommandPreparedStatementUpdate)(nil), // 51: arrow.flight.protocol.sql.CommandPreparedStatementUpdate - (*DoPutUpdateResult)(nil), // 52: arrow.flight.protocol.sql.DoPutUpdateResult - (*ActionCancelQueryRequest)(nil), // 53: arrow.flight.protocol.sql.ActionCancelQueryRequest - (*ActionCancelQueryResult)(nil), // 54: arrow.flight.protocol.sql.ActionCancelQueryResult - (*descriptorpb.MessageOptions)(nil), // 55: google.protobuf.MessageOptions + (SqlInfo)(0), // 0: arrow.flight.protocol.sql.SqlInfo + (SqlSupportedTransaction)(0), // 1: arrow.flight.protocol.sql.SqlSupportedTransaction + (SqlSupportedCaseSensitivity)(0), // 2: arrow.flight.protocol.sql.SqlSupportedCaseSensitivity + (SqlNullOrdering)(0), // 3: arrow.flight.protocol.sql.SqlNullOrdering + (SupportedSqlGrammar)(0), // 4: arrow.flight.protocol.sql.SupportedSqlGrammar + (SupportedAnsi92SqlGrammarLevel)(0), // 5: arrow.flight.protocol.sql.SupportedAnsi92SqlGrammarLevel + (SqlOuterJoinsSupportLevel)(0), // 6: arrow.flight.protocol.sql.SqlOuterJoinsSupportLevel + (SqlSupportedGroupBy)(0), // 7: arrow.flight.protocol.sql.SqlSupportedGroupBy + (SqlSupportedElementActions)(0), // 8: arrow.flight.protocol.sql.SqlSupportedElementActions + (SqlSupportedPositionedCommands)(0), // 9: arrow.flight.protocol.sql.SqlSupportedPositionedCommands + (SqlSupportedSubqueries)(0), // 10: arrow.flight.protocol.sql.SqlSupportedSubqueries + (SqlSupportedUnions)(0), // 11: arrow.flight.protocol.sql.SqlSupportedUnions + (SqlTransactionIsolationLevel)(0), // 12: arrow.flight.protocol.sql.SqlTransactionIsolationLevel + (SqlSupportedTransactions)(0), // 13: arrow.flight.protocol.sql.SqlSupportedTransactions + (SqlSupportedResultSetType)(0), // 14: arrow.flight.protocol.sql.SqlSupportedResultSetType + (SqlSupportedResultSetConcurrency)(0), // 15: arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency + (SqlSupportsConvert)(0), // 16: arrow.flight.protocol.sql.SqlSupportsConvert + (XdbcDataType)(0), // 17: arrow.flight.protocol.sql.XdbcDataType + (XdbcDatetimeSubcode)(0), // 18: arrow.flight.protocol.sql.XdbcDatetimeSubcode + (Nullable)(0), // 19: arrow.flight.protocol.sql.Nullable + (Searchable)(0), // 20: arrow.flight.protocol.sql.Searchable + (UpdateDeleteRules)(0), // 21: arrow.flight.protocol.sql.UpdateDeleteRules + (ActionEndTransactionRequest_EndTransaction)(0), // 22: arrow.flight.protocol.sql.ActionEndTransactionRequest.EndTransaction + (ActionEndSavepointRequest_EndSavepoint)(0), // 23: arrow.flight.protocol.sql.ActionEndSavepointRequest.EndSavepoint + (CommandStatementIngest_TableDefinitionOptions_TableNotExistOption)(0), // 24: arrow.flight.protocol.sql.CommandStatementIngest.TableDefinitionOptions.TableNotExistOption + (CommandStatementIngest_TableDefinitionOptions_TableExistsOption)(0), // 25: arrow.flight.protocol.sql.CommandStatementIngest.TableDefinitionOptions.TableExistsOption + (ActionCancelQueryResult_CancelResult)(0), // 26: arrow.flight.protocol.sql.ActionCancelQueryResult.CancelResult + (*CommandGetSqlInfo)(nil), // 27: arrow.flight.protocol.sql.CommandGetSqlInfo + (*CommandGetXdbcTypeInfo)(nil), // 28: arrow.flight.protocol.sql.CommandGetXdbcTypeInfo + (*CommandGetCatalogs)(nil), // 29: arrow.flight.protocol.sql.CommandGetCatalogs + (*CommandGetDbSchemas)(nil), // 30: arrow.flight.protocol.sql.CommandGetDbSchemas + (*CommandGetTables)(nil), // 31: arrow.flight.protocol.sql.CommandGetTables + (*CommandGetTableTypes)(nil), // 32: arrow.flight.protocol.sql.CommandGetTableTypes + (*CommandGetPrimaryKeys)(nil), // 33: arrow.flight.protocol.sql.CommandGetPrimaryKeys + (*CommandGetExportedKeys)(nil), // 34: arrow.flight.protocol.sql.CommandGetExportedKeys + (*CommandGetImportedKeys)(nil), // 35: arrow.flight.protocol.sql.CommandGetImportedKeys + (*CommandGetCrossReference)(nil), // 36: arrow.flight.protocol.sql.CommandGetCrossReference + (*ActionCreatePreparedStatementRequest)(nil), // 37: arrow.flight.protocol.sql.ActionCreatePreparedStatementRequest + (*SubstraitPlan)(nil), // 38: arrow.flight.protocol.sql.SubstraitPlan + (*ActionCreatePreparedSubstraitPlanRequest)(nil), // 39: arrow.flight.protocol.sql.ActionCreatePreparedSubstraitPlanRequest + (*ActionCreatePreparedStatementResult)(nil), // 40: arrow.flight.protocol.sql.ActionCreatePreparedStatementResult + (*ActionClosePreparedStatementRequest)(nil), // 41: arrow.flight.protocol.sql.ActionClosePreparedStatementRequest + (*ActionBeginTransactionRequest)(nil), // 42: arrow.flight.protocol.sql.ActionBeginTransactionRequest + (*ActionBeginSavepointRequest)(nil), // 43: arrow.flight.protocol.sql.ActionBeginSavepointRequest + (*ActionBeginTransactionResult)(nil), // 44: arrow.flight.protocol.sql.ActionBeginTransactionResult + (*ActionBeginSavepointResult)(nil), // 45: arrow.flight.protocol.sql.ActionBeginSavepointResult + (*ActionEndTransactionRequest)(nil), // 46: arrow.flight.protocol.sql.ActionEndTransactionRequest + (*ActionEndSavepointRequest)(nil), // 47: arrow.flight.protocol.sql.ActionEndSavepointRequest + (*CommandStatementQuery)(nil), // 48: arrow.flight.protocol.sql.CommandStatementQuery + (*CommandStatementSubstraitPlan)(nil), // 49: arrow.flight.protocol.sql.CommandStatementSubstraitPlan + (*TicketStatementQuery)(nil), // 50: arrow.flight.protocol.sql.TicketStatementQuery + (*CommandPreparedStatementQuery)(nil), // 51: arrow.flight.protocol.sql.CommandPreparedStatementQuery + (*CommandStatementUpdate)(nil), // 52: arrow.flight.protocol.sql.CommandStatementUpdate + (*CommandPreparedStatementUpdate)(nil), // 53: arrow.flight.protocol.sql.CommandPreparedStatementUpdate + (*CommandStatementIngest)(nil), // 54: arrow.flight.protocol.sql.CommandStatementIngest + (*DoPutUpdateResult)(nil), // 55: arrow.flight.protocol.sql.DoPutUpdateResult + (*ActionCancelQueryRequest)(nil), // 56: arrow.flight.protocol.sql.ActionCancelQueryRequest + (*ActionCancelQueryResult)(nil), // 57: arrow.flight.protocol.sql.ActionCancelQueryResult + (*CommandStatementIngest_TableDefinitionOptions)(nil), // 58: arrow.flight.protocol.sql.CommandStatementIngest.TableDefinitionOptions + nil, // 59: arrow.flight.protocol.sql.CommandStatementIngest.OptionsEntry + (*descriptorpb.MessageOptions)(nil), // 60: google.protobuf.MessageOptions } var file_FlightSql_proto_depIdxs = []int32{ - 36, // 0: arrow.flight.protocol.sql.ActionCreatePreparedSubstraitPlanRequest.plan:type_name -> arrow.flight.protocol.sql.SubstraitPlan + 38, // 0: arrow.flight.protocol.sql.ActionCreatePreparedSubstraitPlanRequest.plan:type_name -> arrow.flight.protocol.sql.SubstraitPlan 22, // 1: arrow.flight.protocol.sql.ActionEndTransactionRequest.action:type_name -> arrow.flight.protocol.sql.ActionEndTransactionRequest.EndTransaction 23, // 2: arrow.flight.protocol.sql.ActionEndSavepointRequest.action:type_name -> arrow.flight.protocol.sql.ActionEndSavepointRequest.EndSavepoint - 36, // 3: arrow.flight.protocol.sql.CommandStatementSubstraitPlan.plan:type_name -> arrow.flight.protocol.sql.SubstraitPlan - 24, // 4: arrow.flight.protocol.sql.ActionCancelQueryResult.result:type_name -> arrow.flight.protocol.sql.ActionCancelQueryResult.CancelResult - 55, // 5: arrow.flight.protocol.sql.experimental:extendee -> google.protobuf.MessageOptions - 6, // [6:6] is the sub-list for method output_type - 6, // [6:6] is the sub-list for method input_type - 6, // [6:6] is the sub-list for extension type_name - 5, // [5:6] is the sub-list for extension extendee - 0, // [0:5] is the sub-list for field type_name + 38, // 3: arrow.flight.protocol.sql.CommandStatementSubstraitPlan.plan:type_name -> arrow.flight.protocol.sql.SubstraitPlan + 58, // 4: arrow.flight.protocol.sql.CommandStatementIngest.table_definition_options:type_name -> arrow.flight.protocol.sql.CommandStatementIngest.TableDefinitionOptions + 59, // 5: arrow.flight.protocol.sql.CommandStatementIngest.options:type_name -> arrow.flight.protocol.sql.CommandStatementIngest.OptionsEntry + 26, // 6: arrow.flight.protocol.sql.ActionCancelQueryResult.result:type_name -> arrow.flight.protocol.sql.ActionCancelQueryResult.CancelResult + 24, // 7: arrow.flight.protocol.sql.CommandStatementIngest.TableDefinitionOptions.if_not_exist:type_name -> arrow.flight.protocol.sql.CommandStatementIngest.TableDefinitionOptions.TableNotExistOption + 25, // 8: arrow.flight.protocol.sql.CommandStatementIngest.TableDefinitionOptions.if_exists:type_name -> arrow.flight.protocol.sql.CommandStatementIngest.TableDefinitionOptions.TableExistsOption + 60, // 9: arrow.flight.protocol.sql.experimental:extendee -> google.protobuf.MessageOptions + 10, // [10:10] is the sub-list for method output_type + 10, // [10:10] is the sub-list for method input_type + 10, // [10:10] is the sub-list for extension type_name + 9, // [9:10] is the sub-list for extension extendee + 0, // [0:9] is the sub-list for field type_name } func init() { file_FlightSql_proto_init() } @@ -5532,7 +5713,7 @@ func file_FlightSql_proto_init() { } } file_FlightSql_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DoPutUpdateResult); i { + switch v := v.(*CommandStatementIngest); i { case 0: return &v.state case 1: @@ -5544,7 +5725,7 @@ func file_FlightSql_proto_init() { } } file_FlightSql_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ActionCancelQueryRequest); i { + switch v := v.(*DoPutUpdateResult); i { case 0: return &v.state case 1: @@ -5556,6 +5737,18 @@ func file_FlightSql_proto_init() { } } file_FlightSql_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ActionCancelQueryRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ActionCancelQueryResult); i { case 0: return &v.state @@ -5567,6 +5760,18 @@ func file_FlightSql_proto_init() { return nil } } + file_FlightSql_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandStatementIngest_TableDefinitionOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } file_FlightSql_proto_msgTypes[1].OneofWrappers = []interface{}{} file_FlightSql_proto_msgTypes[3].OneofWrappers = []interface{}{} @@ -5580,13 +5785,14 @@ func file_FlightSql_proto_init() { file_FlightSql_proto_msgTypes[21].OneofWrappers = []interface{}{} file_FlightSql_proto_msgTypes[22].OneofWrappers = []interface{}{} file_FlightSql_proto_msgTypes[25].OneofWrappers = []interface{}{} + file_FlightSql_proto_msgTypes[27].OneofWrappers = []interface{}{} type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_FlightSql_proto_rawDesc, - NumEnums: 25, - NumMessages: 30, + NumEnums: 27, + NumMessages: 33, NumExtensions: 1, NumServices: 0, }, diff --git a/go/arrow/flight/gen/flight/Flight_grpc.pb.go b/go/arrow/flight/gen/flight/Flight_grpc.pb.go index 237cb1fe2df..841a3f26b98 100644 --- a/go/arrow/flight/gen/flight/Flight_grpc.pb.go +++ b/go/arrow/flight/gen/flight/Flight_grpc.pb.go @@ -1,3 +1,20 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +//

+// http://www.apache.org/licenses/LICENSE-2.0 +//

+// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.2.0 @@ -18,6 +35,19 @@ import ( // Requires gRPC-Go v1.32.0 or later. const _ = grpc.SupportPackageIsVersion7 +const ( + FlightService_Handshake_FullMethodName = "/arrow.flight.protocol.FlightService/Handshake" + FlightService_ListFlights_FullMethodName = "/arrow.flight.protocol.FlightService/ListFlights" + FlightService_GetFlightInfo_FullMethodName = "/arrow.flight.protocol.FlightService/GetFlightInfo" + FlightService_PollFlightInfo_FullMethodName = "/arrow.flight.protocol.FlightService/PollFlightInfo" + FlightService_GetSchema_FullMethodName = "/arrow.flight.protocol.FlightService/GetSchema" + FlightService_DoGet_FullMethodName = "/arrow.flight.protocol.FlightService/DoGet" + FlightService_DoPut_FullMethodName = "/arrow.flight.protocol.FlightService/DoPut" + FlightService_DoExchange_FullMethodName = "/arrow.flight.protocol.FlightService/DoExchange" + FlightService_DoAction_FullMethodName = "/arrow.flight.protocol.FlightService/DoAction" + FlightService_ListActions_FullMethodName = "/arrow.flight.protocol.FlightService/ListActions" +) + // FlightServiceClient is the client API for FlightService service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. @@ -113,7 +143,7 @@ func NewFlightServiceClient(cc grpc.ClientConnInterface) FlightServiceClient { } func (c *flightServiceClient) Handshake(ctx context.Context, opts ...grpc.CallOption) (FlightService_HandshakeClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[0], "/arrow.flight.protocol.FlightService/Handshake", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[0], FlightService_Handshake_FullMethodName, opts...) if err != nil { return nil, err } @@ -144,7 +174,7 @@ func (x *flightServiceHandshakeClient) Recv() (*HandshakeResponse, error) { } func (c *flightServiceClient) ListFlights(ctx context.Context, in *Criteria, opts ...grpc.CallOption) (FlightService_ListFlightsClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[1], "/arrow.flight.protocol.FlightService/ListFlights", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[1], FlightService_ListFlights_FullMethodName, opts...) if err != nil { return nil, err } @@ -177,7 +207,7 @@ func (x *flightServiceListFlightsClient) Recv() (*FlightInfo, error) { func (c *flightServiceClient) GetFlightInfo(ctx context.Context, in *FlightDescriptor, opts ...grpc.CallOption) (*FlightInfo, error) { out := new(FlightInfo) - err := c.cc.Invoke(ctx, "/arrow.flight.protocol.FlightService/GetFlightInfo", in, out, opts...) + err := c.cc.Invoke(ctx, FlightService_GetFlightInfo_FullMethodName, in, out, opts...) if err != nil { return nil, err } @@ -186,7 +216,7 @@ func (c *flightServiceClient) GetFlightInfo(ctx context.Context, in *FlightDescr func (c *flightServiceClient) PollFlightInfo(ctx context.Context, in *FlightDescriptor, opts ...grpc.CallOption) (*PollInfo, error) { out := new(PollInfo) - err := c.cc.Invoke(ctx, "/arrow.flight.protocol.FlightService/PollFlightInfo", in, out, opts...) + err := c.cc.Invoke(ctx, FlightService_PollFlightInfo_FullMethodName, in, out, opts...) if err != nil { return nil, err } @@ -195,7 +225,7 @@ func (c *flightServiceClient) PollFlightInfo(ctx context.Context, in *FlightDesc func (c *flightServiceClient) GetSchema(ctx context.Context, in *FlightDescriptor, opts ...grpc.CallOption) (*SchemaResult, error) { out := new(SchemaResult) - err := c.cc.Invoke(ctx, "/arrow.flight.protocol.FlightService/GetSchema", in, out, opts...) + err := c.cc.Invoke(ctx, FlightService_GetSchema_FullMethodName, in, out, opts...) if err != nil { return nil, err } @@ -203,7 +233,7 @@ func (c *flightServiceClient) GetSchema(ctx context.Context, in *FlightDescripto } func (c *flightServiceClient) DoGet(ctx context.Context, in *Ticket, opts ...grpc.CallOption) (FlightService_DoGetClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[2], "/arrow.flight.protocol.FlightService/DoGet", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[2], FlightService_DoGet_FullMethodName, opts...) if err != nil { return nil, err } @@ -235,7 +265,7 @@ func (x *flightServiceDoGetClient) Recv() (*FlightData, error) { } func (c *flightServiceClient) DoPut(ctx context.Context, opts ...grpc.CallOption) (FlightService_DoPutClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[3], "/arrow.flight.protocol.FlightService/DoPut", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[3], FlightService_DoPut_FullMethodName, opts...) if err != nil { return nil, err } @@ -266,7 +296,7 @@ func (x *flightServiceDoPutClient) Recv() (*PutResult, error) { } func (c *flightServiceClient) DoExchange(ctx context.Context, opts ...grpc.CallOption) (FlightService_DoExchangeClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[4], "/arrow.flight.protocol.FlightService/DoExchange", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[4], FlightService_DoExchange_FullMethodName, opts...) if err != nil { return nil, err } @@ -297,7 +327,7 @@ func (x *flightServiceDoExchangeClient) Recv() (*FlightData, error) { } func (c *flightServiceClient) DoAction(ctx context.Context, in *Action, opts ...grpc.CallOption) (FlightService_DoActionClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[5], "/arrow.flight.protocol.FlightService/DoAction", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[5], FlightService_DoAction_FullMethodName, opts...) if err != nil { return nil, err } @@ -329,7 +359,7 @@ func (x *flightServiceDoActionClient) Recv() (*Result, error) { } func (c *flightServiceClient) ListActions(ctx context.Context, in *Empty, opts ...grpc.CallOption) (FlightService_ListActionsClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[6], "/arrow.flight.protocol.FlightService/ListActions", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[6], FlightService_ListActions_FullMethodName, opts...) if err != nil { return nil, err } @@ -551,7 +581,7 @@ func _FlightService_GetFlightInfo_Handler(srv interface{}, ctx context.Context, } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/arrow.flight.protocol.FlightService/GetFlightInfo", + FullMethod: FlightService_GetFlightInfo_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(FlightServiceServer).GetFlightInfo(ctx, req.(*FlightDescriptor)) @@ -569,7 +599,7 @@ func _FlightService_PollFlightInfo_Handler(srv interface{}, ctx context.Context, } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/arrow.flight.protocol.FlightService/PollFlightInfo", + FullMethod: FlightService_PollFlightInfo_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(FlightServiceServer).PollFlightInfo(ctx, req.(*FlightDescriptor)) @@ -587,7 +617,7 @@ func _FlightService_GetSchema_Handler(srv interface{}, ctx context.Context, dec } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/arrow.flight.protocol.FlightService/GetSchema", + FullMethod: FlightService_GetSchema_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(FlightServiceServer).GetSchema(ctx, req.(*FlightDescriptor)) diff --git a/go/arrow/internal/flight_integration/scenario.go b/go/arrow/internal/flight_integration/scenario.go index edddd201dda..504726e024e 100644 --- a/go/arrow/internal/flight_integration/scenario.go +++ b/go/arrow/internal/flight_integration/scenario.go @@ -82,6 +82,8 @@ func GetScenario(name string, args ...string) Scenario { return &flightSqlExtensionScenarioTester{} case "session_options": return &sessionOptionsScenarioTester{} + case "flight_sql:ingestion": + return &flightSqlIngestionScenarioTester{} case "": if len(args) > 0 { return &defaultIntegrationTester{path: args[0]} @@ -1358,6 +1360,7 @@ const ( updateStatementWithTransactionExpectedRows int64 = 15000 updatePreparedStatementExpectedRows int64 = 20000 updatePreparedStatementWithTransactionExpectedRows int64 = 25000 + ingestStatementExpectedRows int64 = 3 ) type flightSqlScenarioTester struct { @@ -2948,3 +2951,136 @@ func (tester *sessionOptionsScenarioTester) ValidateSeventhGetSessionOptions(ctx return nil } + +type flightSqlIngestionScenarioTester struct { + flightsql.BaseServer +} + +func (m *flightSqlIngestionScenarioTester) MakeServer(port int) flight.Server { + srv := flight.NewServerWithMiddleware(nil) + m.RegisterSqlInfo(flightsql.SqlInfoFlightSqlServerBulkIngestion, true) + m.RegisterSqlInfo(flightsql.SqlInfoFlightSqlServerIngestTransactionsSupported, true) + + srv.RegisterFlightService(flightsql.NewFlightServer(m)) + initServer(port, srv) + return srv +} + +func (m *flightSqlIngestionScenarioTester) RunClient(addr string, opts ...grpc.DialOption) error { + client, err := flightsql.NewClient(addr, nil, nil, opts...) + if err != nil { + return err + } + defer client.Close() + + return m.ValidateIngestion(client) +} + +func (m *flightSqlIngestionScenarioTester) ValidateIngestion(client *flightsql.Client) error { + ctx := context.Background() + opts := getIngestOptions() + ingestResult, err := client.ExecuteIngest(ctx, getIngestRecords(), &opts) + if err != nil { + return err + } + if ingestResult != ingestStatementExpectedRows { + return fmt.Errorf("expected ingest return %d got %d", ingestStatementExpectedRows, ingestResult) + } + return nil +} + +func (m *flightSqlIngestionScenarioTester) DoPutCommandStatementIngest(ctx context.Context, cmd flightsql.StatementIngest, rdr flight.MessageReader) (int64, error) { + expectedSchema := getIngestSchema() + expectedOpts := getIngestOptions() + + if err := assertEq(expectedOpts.TableDefinitionOptions.IfExists, cmd.GetTableDefinitionOptions().IfExists); err != nil { + return 0, err + } + + if err := assertEq(expectedOpts.TableDefinitionOptions.IfNotExist, cmd.GetTableDefinitionOptions().IfNotExist); err != nil { + return 0, err + } + + if err := assertEq(expectedOpts.Table, cmd.GetTable()); err != nil { + return 0, err + } + + if err := assertEq(*expectedOpts.Schema, cmd.GetSchema()); err != nil { + return 0, err + } + + if err := assertEq(*expectedOpts.Catalog, cmd.GetCatalog()); err != nil { + return 0, err + } + + if err := assertEq(expectedOpts.Temporary, cmd.GetTemporary()); err != nil { + return 0, err + } + + if err := assertEq(expectedOpts.TransactionId, cmd.GetTransactionId()); err != nil { + return 0, err + } + + if err := assertEq(expectedOpts.Options, cmd.GetOptions()); err != nil { + return 0, err + } + + var nRecords int64 + for rdr.Next() { + rec := rdr.Record() + nRecords += rec.NumRows() + + if err := assertEq(true, expectedSchema.Equal(rec.Schema())); err != nil { + return 0, err + } + } + + return nRecords, nil +} + +// Options to assert before/after mocked ingest call +func getIngestOptions() flightsql.ExecuteIngestOpts { + tableDefinitionOptions := flightsql.TableDefinitionOptions{ + IfNotExist: flightsql.TableDefinitionOptionsTableNotExistOptionCreate, + IfExists: flightsql.TableDefinitionOptionsTableExistsOptionReplace, + } + table := "test_table" + schema := "test_schema" + catalog := "test_catalog" + temporary := true + transactionId := []byte("123") + options := map[string]string{ + "key1": "val1", + "key2": "val2", + } + + return flightsql.ExecuteIngestOpts{ + TableDefinitionOptions: &tableDefinitionOptions, + Table: table, + Schema: &schema, + Catalog: &catalog, + Temporary: temporary, + TransactionId: transactionId, + Options: options, + } +} + +// Schema for ingest records; asserted on records received by handler +func getIngestSchema() *arrow.Schema { + return arrow.NewSchema([]arrow.Field{{Name: "test_field", Type: arrow.PrimitiveTypes.Int64, Nullable: true}}, nil) +} + +// Prepare records for ingestion with known length and schema +func getIngestRecords() array.RecordReader { + schema := getIngestSchema() + + arr := array.MakeArrayOfNull(memory.DefaultAllocator, arrow.PrimitiveTypes.Int64, int(ingestStatementExpectedRows)) + defer arr.Release() + + rec := array.NewRecord(schema, []arrow.Array{arr}, ingestStatementExpectedRows) + defer rec.Release() + + rdr, _ := array.NewRecordReader(schema, []arrow.Record{rec}) + + return rdr +} From a0c7497de24bf65a188d01304d31015c16075ec1 Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky Date: Sun, 17 Mar 2024 17:49:10 -0400 Subject: [PATCH 02/13] regenerate go pb --- go/arrow/flight/gen/flight/Flight.pb.go | 16 +- go/arrow/flight/gen/flight/FlightSql.pb.go | 258 ++++++++++++++++--- go/arrow/flight/gen/flight/Flight_grpc.pb.go | 58 +---- 3 files changed, 248 insertions(+), 84 deletions(-) diff --git a/go/arrow/flight/gen/flight/Flight.pb.go b/go/arrow/flight/gen/flight/Flight.pb.go index 42d4493c8f3..d9477ee062f 100644 --- a/go/arrow/flight/gen/flight/Flight.pb.go +++ b/go/arrow/flight/gen/flight/Flight.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.31.0 -// protoc v4.25.2 +// protoc v4.25.3 // source: Flight.proto package flight @@ -1210,11 +1210,15 @@ type FlightEndpoint struct { // be redeemed on the current service where the ticket was // generated. // - // If the list is not empty, the expectation is that the ticket can - // be redeemed at any of the locations, and that the data returned - // will be equivalent. In this case, the ticket may only be redeemed - // at one of the given locations, and not (necessarily) on the - // current service. + // If the list is not empty, the expectation is that the ticket can be + // redeemed at any of the locations, and that the data returned will be + // equivalent. In this case, the ticket may only be redeemed at one of the + // given locations, and not (necessarily) on the current service. If one + // of the given locations is "arrow-flight-reuse-connection://?", the + // client may redeem the ticket on the service where the ticket was + // generated (i.e., the same as above), in addition to the other + // locations. (This URI was chosen to maximize compatibility, as 'scheme:' + // or 'scheme://' are not accepted by Java's java.net.URI.) // // In other words, an application can use multiple locations to // represent redundant and/or load balanced services. diff --git a/go/arrow/flight/gen/flight/FlightSql.pb.go b/go/arrow/flight/gen/flight/FlightSql.pb.go index 2a230591a2e..3ff7c5fc147 100644 --- a/go/arrow/flight/gen/flight/FlightSql.pb.go +++ b/go/arrow/flight/gen/flight/FlightSql.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.31.0 -// protoc v4.25.2 +// protoc v4.25.3 // source: FlightSql.proto package flight @@ -81,6 +81,17 @@ const ( // Retrieves a boolean value indicating whether the Flight SQL Server supports explicit // query cancellation (the CancelQuery action). SqlInfo_FLIGHT_SQL_SERVER_CANCEL SqlInfo = 9 + // Retrieves a boolean value indicating whether the Flight SQL Server supports executing + // bulk ingestion. + SqlInfo_FLIGHT_SQL_SERVER_BULK_INGESTION SqlInfo = 10 + // Retrieves a boolean value indicating whether transactions are supported for bulk ingestion. If not, invoking + // the method commit in the context of a bulk ingestion is a noop, and the isolation level is + // `arrow.flight.protocol.sql.SqlTransactionIsolationLevel.TRANSACTION_NONE`. + // + // Returns: + // - false: if bulk ingestion transactions are unsupported; + // - true: if bulk ingestion transactions are supported. + SqlInfo_FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED SqlInfo = 11 // Retrieves an int32 indicating the timeout (in milliseconds) for prepared statement handles. // // If 0, there is no timeout. Servers should reset the timeout when the handle is used in a command. @@ -4298,9 +4309,117 @@ func (x *CommandPreparedStatementUpdate) GetPreparedStatementHandle() []byte { return nil } -// Returned from the RPC call DoPut when a CommandStatementUpdate -// CommandPreparedStatementUpdate was in the request, containing -// results from the update. +// Represents a bulk ingestion request. Used in the command member of FlightDescriptor +// for the the RPC call DoPut to cause the server load the contents of the stream's +// FlightData into the target destination. +type CommandStatementIngest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The behavior for handling the table definition. + TableDefinitionOptions *CommandStatementIngest_TableDefinitionOptions `protobuf:"bytes,1,opt,name=table_definition_options,json=tableDefinitionOptions,proto3" json:"table_definition_options,omitempty"` + // The table to load data into. + Table string `protobuf:"bytes,2,opt,name=table,proto3" json:"table,omitempty"` + // The db_schema of the destination table to load data into. If unset, a backend-specific default may be used. + Schema *string `protobuf:"bytes,3,opt,name=schema,proto3,oneof" json:"schema,omitempty"` + // The catalog of the destination table to load data into. If unset, a backend-specific default may be used. + Catalog *string `protobuf:"bytes,4,opt,name=catalog,proto3,oneof" json:"catalog,omitempty"` + // Store ingested data in a temporary table. + // The effect of setting temporary is to place the table in a backend-defined namespace, and to drop the table at the end of the session. + // The namespacing may make use of a backend-specific schema and/or catalog. + // The server should return an error if an explicit choice of schema or catalog is incompatible with the server's namespacing decision. + Temporary bool `protobuf:"varint,5,opt,name=temporary,proto3" json:"temporary,omitempty"` + // Perform the ingestion as part of this transaction. If specified, results should not be committed in the event of an error/cancellation. + TransactionId []byte `protobuf:"bytes,6,opt,name=transaction_id,json=transactionId,proto3,oneof" json:"transaction_id,omitempty"` + // Backend-specific options. + Options map[string]string `protobuf:"bytes,1000,rep,name=options,proto3" json:"options,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *CommandStatementIngest) Reset() { + *x = CommandStatementIngest{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandStatementIngest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandStatementIngest) ProtoMessage() {} + +func (x *CommandStatementIngest) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[27] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CommandStatementIngest.ProtoReflect.Descriptor instead. +func (*CommandStatementIngest) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{27} +} + +func (x *CommandStatementIngest) GetTableDefinitionOptions() *CommandStatementIngest_TableDefinitionOptions { + if x != nil { + return x.TableDefinitionOptions + } + return nil +} + +func (x *CommandStatementIngest) GetTable() string { + if x != nil { + return x.Table + } + return "" +} + +func (x *CommandStatementIngest) GetSchema() string { + if x != nil && x.Schema != nil { + return *x.Schema + } + return "" +} + +func (x *CommandStatementIngest) GetCatalog() string { + if x != nil && x.Catalog != nil { + return *x.Catalog + } + return "" +} + +func (x *CommandStatementIngest) GetTemporary() bool { + if x != nil { + return x.Temporary + } + return false +} + +func (x *CommandStatementIngest) GetTransactionId() []byte { + if x != nil { + return x.TransactionId + } + return nil +} + +func (x *CommandStatementIngest) GetOptions() map[string]string { + if x != nil { + return x.Options + } + return nil +} + +// Returned from the RPC call DoPut when a CommandStatementUpdate, +// CommandPreparedStatementUpdate, or CommandStatementIngest was +// in the request, containing results from the update. type DoPutUpdateResult struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -4776,37 +4895,108 @@ var file_FlightSql_proto_rawDesc = []byte{ 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x68, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x17, 0x70, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x64, 0x53, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x3a, 0x03, - 0xc0, 0x3e, 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x44, 0x6f, 0x50, 0x75, 0x74, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x72, 0x65, 0x63, 0x6f, - 0x72, 0x64, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, - 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x3a, 0x03, 0xc0, 0x3e, 0x01, - 0x22, 0x35, 0x0a, 0x18, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, - 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, - 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, - 0x3a, 0x05, 0xc0, 0x3e, 0x01, 0x18, 0x01, 0x22, 0x87, 0x02, 0x0a, 0x17, 0x41, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x12, 0x57, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0e, 0x32, 0x3f, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, - 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x73, 0x71, 0x6c, 0x2e, + 0xc0, 0x3e, 0x01, 0x22, 0xb1, 0x08, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x67, 0x65, 0x73, 0x74, 0x12, 0x82, + 0x01, 0x0a, 0x18, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x48, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x73, 0x71, 0x6c, 0x2e, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, + 0x67, 0x65, 0x73, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x16, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1b, 0x0a, 0x06, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x06, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x1d, 0x0a, 0x07, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, + 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x07, 0x63, 0x61, 0x74, 0x61, 0x6c, + 0x6f, 0x67, 0x88, 0x01, 0x01, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, + 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, + 0x61, 0x72, 0x79, 0x12, 0x2a, 0x0a, 0x0e, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x02, 0x52, 0x0d, 0x74, + 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, + 0x59, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xe8, 0x07, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x3e, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x73, 0x71, 0x6c, 0x2e, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, + 0x67, 0x65, 0x73, 0x74, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0xaf, 0x04, 0x0a, 0x16, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x7e, 0x0a, 0x0c, 0x69, 0x66, 0x5f, 0x6e, 0x6f, 0x74, 0x5f, + 0x65, 0x78, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x5c, 0x2e, 0x61, 0x72, + 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x63, 0x6f, 0x6c, 0x2e, 0x73, 0x71, 0x6c, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x67, 0x65, 0x73, 0x74, 0x2e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x6f, 0x74, 0x45, 0x78, + 0x69, 0x73, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x69, 0x66, 0x4e, 0x6f, 0x74, + 0x45, 0x78, 0x69, 0x73, 0x74, 0x12, 0x77, 0x0a, 0x09, 0x69, 0x66, 0x5f, 0x65, 0x78, 0x69, 0x73, + 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x5a, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, + 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, + 0x2e, 0x73, 0x71, 0x6c, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x67, 0x65, 0x73, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x69, 0x66, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x22, 0x81, + 0x01, 0x0a, 0x13, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x6f, 0x74, 0x45, 0x78, 0x69, 0x73, 0x74, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x26, 0x0a, 0x22, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, + 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, 0x5f, 0x4f, 0x50, 0x54, 0x49, 0x4f, 0x4e, + 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x21, + 0x0a, 0x1d, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x58, 0x49, 0x53, + 0x54, 0x5f, 0x4f, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x10, + 0x01, 0x12, 0x1f, 0x0a, 0x1b, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x45, + 0x58, 0x49, 0x53, 0x54, 0x5f, 0x4f, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, 0x49, 0x4c, + 0x10, 0x02, 0x22, 0x97, 0x01, 0x0a, 0x11, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, + 0x74, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x1f, 0x54, 0x41, 0x42, 0x4c, + 0x45, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, 0x53, 0x5f, 0x4f, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1c, 0x0a, + 0x18, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, 0x53, 0x5f, 0x4f, 0x50, + 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x10, 0x01, 0x12, 0x1e, 0x0a, 0x1a, 0x54, + 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, 0x53, 0x5f, 0x4f, 0x50, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x02, 0x12, 0x1f, 0x0a, 0x1b, 0x54, + 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, 0x53, 0x5f, 0x4f, 0x50, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x50, 0x4c, 0x41, 0x43, 0x45, 0x10, 0x03, 0x1a, 0x3a, 0x0a, 0x0c, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x3a, 0x03, 0xc0, 0x3e, 0x01, 0x42, 0x09, 0x0a, + 0x07, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x63, 0x61, 0x74, + 0x61, 0x6c, 0x6f, 0x67, 0x42, 0x11, 0x0a, 0x0f, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x22, 0x3b, 0x0a, 0x11, 0x44, 0x6f, 0x50, 0x75, 0x74, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x21, 0x0a, 0x0c, + 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0b, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x3a, + 0x03, 0xc0, 0x3e, 0x01, 0x22, 0x35, 0x0a, 0x18, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x12, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, + 0x69, 0x6e, 0x66, 0x6f, 0x3a, 0x05, 0xc0, 0x3e, 0x01, 0x18, 0x01, 0x22, 0x87, 0x02, 0x0a, 0x17, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x51, 0x75, 0x65, 0x72, - 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x8b, 0x01, 0x0a, - 0x0c, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x1d, 0x0a, - 0x19, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x55, - 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1b, 0x0a, 0x17, - 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x43, 0x41, - 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x01, 0x12, 0x1c, 0x0a, 0x18, 0x43, 0x41, 0x4e, - 0x43, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, - 0x4c, 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x21, 0x0a, 0x1d, 0x43, 0x41, 0x4e, 0x43, 0x45, - 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x43, 0x41, 0x4e, - 0x43, 0x45, 0x4c, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x03, 0x3a, 0x05, 0xc0, 0x3e, 0x01, 0x18, - 0x01, 0x2a, 0xb7, 0x18, 0x0a, 0x07, 0x53, 0x71, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1a, 0x0a, - 0x16, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, - 0x45, 0x52, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x46, 0x4c, 0x49, - 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x56, - 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x12, 0x23, 0x0a, 0x1f, 0x46, 0x4c, 0x49, 0x47, - 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x41, 0x52, - 0x52, 0x4f, 0x57, 0x5f, 0x56, 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x02, 0x12, 0x1f, 0x0a, + 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x57, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x3f, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, + 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, + 0x73, 0x71, 0x6c, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x43, 0x61, 0x6e, 0x63, + 0x65, 0x6c, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x22, 0x8b, 0x01, 0x0a, 0x0c, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x52, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x12, 0x1d, 0x0a, 0x19, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, + 0x4c, 0x54, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, + 0x12, 0x1b, 0x0a, 0x17, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, + 0x54, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x01, 0x12, 0x1c, 0x0a, + 0x18, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x43, + 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x21, 0x0a, 0x1d, 0x43, + 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x5f, 0x4e, 0x4f, 0x54, + 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x03, 0x3a, 0x05, + 0xc0, 0x3e, 0x01, 0x18, 0x01, 0x2a, 0x92, 0x19, 0x0a, 0x07, 0x53, 0x71, 0x6c, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x1a, 0x0a, 0x16, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, + 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x10, 0x00, 0x12, 0x1d, 0x0a, + 0x19, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, + 0x45, 0x52, 0x5f, 0x56, 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x12, 0x23, 0x0a, 0x1f, + 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, + 0x52, 0x5f, 0x41, 0x52, 0x52, 0x4f, 0x57, 0x5f, 0x56, 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, + 0x02, 0x12, 0x1f, 0x0a, 0x1b, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, + 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x5f, 0x4f, 0x4e, 0x4c, 0x59, + 0x10, 0x03, 0x12, 0x19, 0x0a, 0x15, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, + 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x53, 0x51, 0x4c, 0x10, 0x04, 0x12, 0x1f, 0x0a, 0x1b, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x54, 0x52, 0x41, 0x49, 0x54, 0x10, 0x05, 0x12, 0x2b, 0x0a, 0x27, 0x46, 0x4c, 0x49, 0x47, 0x48, 0x54, 0x5f, 0x53, 0x51, 0x4c, 0x5f, 0x53, 0x45, 0x52, diff --git a/go/arrow/flight/gen/flight/Flight_grpc.pb.go b/go/arrow/flight/gen/flight/Flight_grpc.pb.go index 841a3f26b98..11bbb00131d 100644 --- a/go/arrow/flight/gen/flight/Flight_grpc.pb.go +++ b/go/arrow/flight/gen/flight/Flight_grpc.pb.go @@ -1,24 +1,7 @@ -// -// 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. - // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.2.0 -// - protoc v4.25.2 +// - protoc v4.25.3 // source: Flight.proto package flight @@ -35,19 +18,6 @@ import ( // Requires gRPC-Go v1.32.0 or later. const _ = grpc.SupportPackageIsVersion7 -const ( - FlightService_Handshake_FullMethodName = "/arrow.flight.protocol.FlightService/Handshake" - FlightService_ListFlights_FullMethodName = "/arrow.flight.protocol.FlightService/ListFlights" - FlightService_GetFlightInfo_FullMethodName = "/arrow.flight.protocol.FlightService/GetFlightInfo" - FlightService_PollFlightInfo_FullMethodName = "/arrow.flight.protocol.FlightService/PollFlightInfo" - FlightService_GetSchema_FullMethodName = "/arrow.flight.protocol.FlightService/GetSchema" - FlightService_DoGet_FullMethodName = "/arrow.flight.protocol.FlightService/DoGet" - FlightService_DoPut_FullMethodName = "/arrow.flight.protocol.FlightService/DoPut" - FlightService_DoExchange_FullMethodName = "/arrow.flight.protocol.FlightService/DoExchange" - FlightService_DoAction_FullMethodName = "/arrow.flight.protocol.FlightService/DoAction" - FlightService_ListActions_FullMethodName = "/arrow.flight.protocol.FlightService/ListActions" -) - // FlightServiceClient is the client API for FlightService service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. @@ -143,7 +113,7 @@ func NewFlightServiceClient(cc grpc.ClientConnInterface) FlightServiceClient { } func (c *flightServiceClient) Handshake(ctx context.Context, opts ...grpc.CallOption) (FlightService_HandshakeClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[0], FlightService_Handshake_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[0], "/arrow.flight.protocol.FlightService/Handshake", opts...) if err != nil { return nil, err } @@ -174,7 +144,7 @@ func (x *flightServiceHandshakeClient) Recv() (*HandshakeResponse, error) { } func (c *flightServiceClient) ListFlights(ctx context.Context, in *Criteria, opts ...grpc.CallOption) (FlightService_ListFlightsClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[1], FlightService_ListFlights_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[1], "/arrow.flight.protocol.FlightService/ListFlights", opts...) if err != nil { return nil, err } @@ -207,7 +177,7 @@ func (x *flightServiceListFlightsClient) Recv() (*FlightInfo, error) { func (c *flightServiceClient) GetFlightInfo(ctx context.Context, in *FlightDescriptor, opts ...grpc.CallOption) (*FlightInfo, error) { out := new(FlightInfo) - err := c.cc.Invoke(ctx, FlightService_GetFlightInfo_FullMethodName, in, out, opts...) + err := c.cc.Invoke(ctx, "/arrow.flight.protocol.FlightService/GetFlightInfo", in, out, opts...) if err != nil { return nil, err } @@ -216,7 +186,7 @@ func (c *flightServiceClient) GetFlightInfo(ctx context.Context, in *FlightDescr func (c *flightServiceClient) PollFlightInfo(ctx context.Context, in *FlightDescriptor, opts ...grpc.CallOption) (*PollInfo, error) { out := new(PollInfo) - err := c.cc.Invoke(ctx, FlightService_PollFlightInfo_FullMethodName, in, out, opts...) + err := c.cc.Invoke(ctx, "/arrow.flight.protocol.FlightService/PollFlightInfo", in, out, opts...) if err != nil { return nil, err } @@ -225,7 +195,7 @@ func (c *flightServiceClient) PollFlightInfo(ctx context.Context, in *FlightDesc func (c *flightServiceClient) GetSchema(ctx context.Context, in *FlightDescriptor, opts ...grpc.CallOption) (*SchemaResult, error) { out := new(SchemaResult) - err := c.cc.Invoke(ctx, FlightService_GetSchema_FullMethodName, in, out, opts...) + err := c.cc.Invoke(ctx, "/arrow.flight.protocol.FlightService/GetSchema", in, out, opts...) if err != nil { return nil, err } @@ -233,7 +203,7 @@ func (c *flightServiceClient) GetSchema(ctx context.Context, in *FlightDescripto } func (c *flightServiceClient) DoGet(ctx context.Context, in *Ticket, opts ...grpc.CallOption) (FlightService_DoGetClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[2], FlightService_DoGet_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[2], "/arrow.flight.protocol.FlightService/DoGet", opts...) if err != nil { return nil, err } @@ -265,7 +235,7 @@ func (x *flightServiceDoGetClient) Recv() (*FlightData, error) { } func (c *flightServiceClient) DoPut(ctx context.Context, opts ...grpc.CallOption) (FlightService_DoPutClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[3], FlightService_DoPut_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[3], "/arrow.flight.protocol.FlightService/DoPut", opts...) if err != nil { return nil, err } @@ -296,7 +266,7 @@ func (x *flightServiceDoPutClient) Recv() (*PutResult, error) { } func (c *flightServiceClient) DoExchange(ctx context.Context, opts ...grpc.CallOption) (FlightService_DoExchangeClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[4], FlightService_DoExchange_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[4], "/arrow.flight.protocol.FlightService/DoExchange", opts...) if err != nil { return nil, err } @@ -327,7 +297,7 @@ func (x *flightServiceDoExchangeClient) Recv() (*FlightData, error) { } func (c *flightServiceClient) DoAction(ctx context.Context, in *Action, opts ...grpc.CallOption) (FlightService_DoActionClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[5], FlightService_DoAction_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[5], "/arrow.flight.protocol.FlightService/DoAction", opts...) if err != nil { return nil, err } @@ -359,7 +329,7 @@ func (x *flightServiceDoActionClient) Recv() (*Result, error) { } func (c *flightServiceClient) ListActions(ctx context.Context, in *Empty, opts ...grpc.CallOption) (FlightService_ListActionsClient, error) { - stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[6], FlightService_ListActions_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[6], "/arrow.flight.protocol.FlightService/ListActions", opts...) if err != nil { return nil, err } @@ -581,7 +551,7 @@ func _FlightService_GetFlightInfo_Handler(srv interface{}, ctx context.Context, } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: FlightService_GetFlightInfo_FullMethodName, + FullMethod: "/arrow.flight.protocol.FlightService/GetFlightInfo", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(FlightServiceServer).GetFlightInfo(ctx, req.(*FlightDescriptor)) @@ -599,7 +569,7 @@ func _FlightService_PollFlightInfo_Handler(srv interface{}, ctx context.Context, } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: FlightService_PollFlightInfo_FullMethodName, + FullMethod: "/arrow.flight.protocol.FlightService/PollFlightInfo", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(FlightServiceServer).PollFlightInfo(ctx, req.(*FlightDescriptor)) @@ -617,7 +587,7 @@ func _FlightService_GetSchema_Handler(srv interface{}, ctx context.Context, dec } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: FlightService_GetSchema_FullMethodName, + FullMethod: "/arrow.flight.protocol.FlightService/GetSchema", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(FlightServiceServer).GetSchema(ctx, req.(*FlightDescriptor)) From afc6bccaba0cb4751a413aa604349c16f2649d96 Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky Date: Mon, 18 Mar 2024 05:42:45 -0400 Subject: [PATCH 03/13] handle edge cases in client and improve testing --- go/arrow/flight/flightsql/client.go | 21 +++++++++- go/arrow/flight/flightsql/server_test.go | 50 ++++++++++++++++++++++-- 2 files changed, 66 insertions(+), 5 deletions(-) diff --git a/go/arrow/flight/flightsql/client.go b/go/arrow/flight/flightsql/client.go index 0ea43765cdd..16d4109df71 100644 --- a/go/arrow/flight/flightsql/client.go +++ b/go/arrow/flight/flightsql/client.go @@ -246,6 +246,8 @@ func (c *Client) ExecuteSubstraitUpdate(ctx context.Context, plan SubstraitPlan, var ErrTooManyPutResults = fmt.Errorf("%w: server sent multiple PutResults, expected one ", arrow.ErrInvalid) // ExecuteIngest is for executing a bulk ingestion and only returns the number of affected rows. +// The provided RecordReader will be retained for the duration of the call, but it is the caller's +// responsibility to release the original reference. func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqOptions *ExecuteIngestOpts, opts ...grpc.CallOption) (n int64, err error) { var ( desc *flight.FlightDescriptor @@ -255,6 +257,17 @@ func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqO updateResult pb.DoPutUpdateResult ) + // Servers cannot infer defaults for these parameters, so we validate the request to ensure they are set. + if reqOptions.TableDefinitionOptions == nil { + return 0, fmt.Errorf("cannot ExecuteIngest: invalid ExecuteIngestOpts, TableDefinitionOptions is required") + } + if reqOptions.Table == "" { + return 0, fmt.Errorf("cannot ExecuteIngest: invalid ExecuteIngestOpts, Table is required") + } + + rdr.Retain() + defer rdr.Release() + cmd := (*pb.CommandStatementIngest)(reqOptions) if desc, err = descForCommand(cmd); err != nil { return @@ -271,7 +284,13 @@ func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqO for rdr.Next() { rec := rdr.Record() - wr.Write(rec) + if err = wr.Write(rec); err != nil { + return + } + } + + if err = rdr.Err(); err != nil { + return } if err = stream.CloseSend(); err != nil { diff --git a/go/arrow/flight/flightsql/server_test.go b/go/arrow/flight/flightsql/server_test.go index 125be8ff8c5..c200ed2af23 100644 --- a/go/arrow/flight/flightsql/server_test.go +++ b/go/arrow/flight/flightsql/server_test.go @@ -212,9 +212,16 @@ func (s *FlightSqlServerSuite) SetupTest() { cl, err := flightsql.NewClient(s.s.Addr().String(), nil, nil, dialOpts...) s.Require().NoError(err) s.cl = cl + + checked := memory.NewCheckedAllocator(s.cl.Alloc) + s.cl.Alloc = checked } func (s *FlightSqlServerSuite) TearDownTest() { + checked, ok := s.cl.Alloc.(*memory.CheckedAllocator) + s.Require().True(ok) + checked.AssertSize(s.T(), 0) + s.Require().NoError(s.cl.Close()) s.cl = nil } @@ -291,6 +298,18 @@ func (s *FlightSqlServerSuite) TestExecutePoll() { s.Len(poll.GetInfo().Endpoint, 2) } +func (s *FlightSqlServerSuite) TestExecuteIngestInvalid() { + reclist := arrdata.Records["primitives"] + + rdr, err := array.NewRecordReader(reclist[0].Schema(), reclist) + s.Require().NoError(err) + defer rdr.Release() + + nRecords, err := s.cl.ExecuteIngest(context.TODO(), rdr, &flightsql.ExecuteIngestOpts{}) + s.Require().Error(err) + s.Equal(int64(0), nRecords) +} + func (s *FlightSqlServerSuite) TestExecuteIngest() { var nRecordsExpected int64 @@ -299,10 +318,23 @@ func (s *FlightSqlServerSuite) TestExecuteIngest() { nRecordsExpected += rec.NumRows() } - rdr, _ := array.NewRecordReader(reclist[0].Schema(), reclist) - nRecords, err := s.cl.ExecuteIngest(context.TODO(), rdr, &flightsql.ExecuteIngestOpts{}) - + rdr, err := array.NewRecordReader(reclist[0].Schema(), reclist) s.Require().NoError(err) + defer rdr.Release() + + nRecords, err := s.cl.ExecuteIngest( + context.TODO(), + rdr, + &flightsql.ExecuteIngestOpts{ + TableDefinitionOptions: &flightsql.TableDefinitionOptions{ + IfNotExist: flightsql.TableDefinitionOptionsTableNotExistOptionCreate, + IfExists: flightsql.TableDefinitionOptionsTableExistsOptionReplace, + }, + Table: "test_table", + }, + ) + s.Require().NoError(err) + s.Equal(nRecordsExpected, nRecords) } @@ -487,7 +519,17 @@ func (s *UnimplementedFlightSqlServerSuite) TestDoGet() { func (s *UnimplementedFlightSqlServerSuite) TestExecuteIngest() { reclist := arrdata.Records["primitives"] rdr, _ := array.NewRecordReader(reclist[0].Schema(), reclist) - info, err := s.cl.ExecuteIngest(context.TODO(), rdr, &flightsql.ExecuteIngestOpts{}) + info, err := s.cl.ExecuteIngest( + context.TODO(), + rdr, + &flightsql.ExecuteIngestOpts{ + TableDefinitionOptions: &flightsql.TableDefinitionOptions{ + IfNotExist: flightsql.TableDefinitionOptionsTableNotExistOptionCreate, + IfExists: flightsql.TableDefinitionOptionsTableExistsOptionReplace, + }, + Table: "test_table", + }, + ) st, ok := status.FromError(err) s.True(ok) s.Equal(codes.Unimplemented, st.Code()) From 4357a358046c98d648aadb9c30bb3a63dd114ab6 Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky Date: Mon, 18 Mar 2024 06:33:22 -0400 Subject: [PATCH 04/13] use different arrdata --- go/arrow/flight/flightsql/server_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/go/arrow/flight/flightsql/server_test.go b/go/arrow/flight/flightsql/server_test.go index c200ed2af23..f8035ab8090 100644 --- a/go/arrow/flight/flightsql/server_test.go +++ b/go/arrow/flight/flightsql/server_test.go @@ -517,7 +517,7 @@ func (s *UnimplementedFlightSqlServerSuite) TestDoGet() { } func (s *UnimplementedFlightSqlServerSuite) TestExecuteIngest() { - reclist := arrdata.Records["primitives"] + reclist := arrdata.Records["nulls"] rdr, _ := array.NewRecordReader(reclist[0].Schema(), reclist) info, err := s.cl.ExecuteIngest( context.TODO(), @@ -533,7 +533,7 @@ func (s *UnimplementedFlightSqlServerSuite) TestExecuteIngest() { st, ok := status.FromError(err) s.True(ok) s.Equal(codes.Unimplemented, st.Code()) - s.Equal(st.Message(), "DoPutCommandStatementIngest not implemented") + s.Equal("DoPutCommandStatementIngest not implemented", st.Message()) s.Zero(info) } From 10bf8dfd5d984e2c2eac832dfbc55929a7cf0874 Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky Date: Mon, 18 Mar 2024 08:48:25 -0400 Subject: [PATCH 05/13] generate own test data --- go/arrow/flight/flightsql/server_test.go | 68 ++++++++++++++++++------ 1 file changed, 52 insertions(+), 16 deletions(-) diff --git a/go/arrow/flight/flightsql/server_test.go b/go/arrow/flight/flightsql/server_test.go index f8035ab8090..414a46d18e9 100644 --- a/go/arrow/flight/flightsql/server_test.go +++ b/go/arrow/flight/flightsql/server_test.go @@ -28,7 +28,6 @@ import ( "github.com/apache/arrow/go/v16/arrow/flight/flightsql" pb "github.com/apache/arrow/go/v16/arrow/flight/gen/flight" "github.com/apache/arrow/go/v16/arrow/flight/session" - "github.com/apache/arrow/go/v16/arrow/internal/arrdata" "github.com/apache/arrow/go/v16/arrow/memory" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -299,30 +298,30 @@ func (s *FlightSqlServerSuite) TestExecutePoll() { } func (s *FlightSqlServerSuite) TestExecuteIngestInvalid() { - reclist := arrdata.Records["primitives"] - - rdr, err := array.NewRecordReader(reclist[0].Schema(), reclist) - s.Require().NoError(err) + reclist := []arrow.Record{} + rdr, err := array.NewRecordReader(arrow.NewSchema([]arrow.Field{}, nil), reclist) + s.NoError(err) defer rdr.Release() + // Cannot execute ingest without specifying required options nRecords, err := s.cl.ExecuteIngest(context.TODO(), rdr, &flightsql.ExecuteIngestOpts{}) - s.Require().Error(err) + s.Error(err) s.Equal(int64(0), nRecords) } func (s *FlightSqlServerSuite) TestExecuteIngest() { - var nRecordsExpected int64 - - reclist := arrdata.Records["primitives"] + nRecords := 3 + nRowsPerRecord := 5 + reclist := generateRecords(s.cl.Alloc, nRecords, nRowsPerRecord) for _, rec := range reclist { - nRecordsExpected += rec.NumRows() + defer rec.Release() } rdr, err := array.NewRecordReader(reclist[0].Schema(), reclist) - s.Require().NoError(err) + s.NoError(err) defer rdr.Release() - nRecords, err := s.cl.ExecuteIngest( + nRowsIngested, err := s.cl.ExecuteIngest( context.TODO(), rdr, &flightsql.ExecuteIngestOpts{ @@ -333,9 +332,37 @@ func (s *FlightSqlServerSuite) TestExecuteIngest() { Table: "test_table", }, ) - s.Require().NoError(err) + s.NoError(err) + + nRowsExpected := int64(nRecords * nRowsPerRecord) + s.Equal(nRowsExpected, nRowsIngested) +} + +func generateRecords(alloc memory.Allocator, nRecords, nRowsPerRecord int) []arrow.Record { + schema := arrow.NewSchema( + []arrow.Field{ + {Name: "one", Type: arrow.FixedWidthTypes.Boolean}, + {Name: "two", Type: arrow.BinaryTypes.String}, + {Name: "three", Type: arrow.PrimitiveTypes.Int64}, + }, + nil, + ) - s.Equal(nRecordsExpected, nRecords) + bldr := array.NewRecordBuilder(alloc, schema) + defer bldr.Release() + + var val int + reclist := make([]arrow.Record, nRecords) + for i := 0; i < nRecords; i++ { + for j := 0; j < nRowsPerRecord; j++ { + bldr.Field(0).(*array.BooleanBuilder).Append(val%2 == 0) + bldr.Field(1).(*array.StringBuilder).Append(fmt.Sprint(val)) + bldr.Field(2).(*array.Int64Builder).Append(int64(val)) + val++ + } + reclist[i] = bldr.NewRecord() + } + return reclist } type UnimplementedFlightSqlServerSuite struct { @@ -517,8 +544,17 @@ func (s *UnimplementedFlightSqlServerSuite) TestDoGet() { } func (s *UnimplementedFlightSqlServerSuite) TestExecuteIngest() { - reclist := arrdata.Records["nulls"] - rdr, _ := array.NewRecordReader(reclist[0].Schema(), reclist) + nRecords := 3 + nRowsPerRecord := 5 + reclist := generateRecords(s.cl.Alloc, nRecords, nRowsPerRecord) + for _, rec := range reclist { + defer rec.Release() + } + + rdr, err := array.NewRecordReader(reclist[0].Schema(), reclist) + s.NoError(err) + defer rdr.Release() + info, err := s.cl.ExecuteIngest( context.TODO(), rdr, From 5c0f50e4f98e376d69d139f3baf566334028e861 Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky Date: Mon, 18 Mar 2024 11:22:09 -0400 Subject: [PATCH 06/13] explict return --- go/arrow/flight/flightsql/client.go | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/go/arrow/flight/flightsql/client.go b/go/arrow/flight/flightsql/client.go index 16d4109df71..315c5b82c10 100644 --- a/go/arrow/flight/flightsql/client.go +++ b/go/arrow/flight/flightsql/client.go @@ -248,8 +248,9 @@ var ErrTooManyPutResults = fmt.Errorf("%w: server sent multiple PutResults, expe // ExecuteIngest is for executing a bulk ingestion and only returns the number of affected rows. // The provided RecordReader will be retained for the duration of the call, but it is the caller's // responsibility to release the original reference. -func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqOptions *ExecuteIngestOpts, opts ...grpc.CallOption) (n int64, err error) { +func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqOptions *ExecuteIngestOpts, opts ...grpc.CallOption) (int64, error) { var ( + err error desc *flight.FlightDescriptor stream pb.FlightService_DoPutClient wr *flight.Writer @@ -270,11 +271,11 @@ func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqO cmd := (*pb.CommandStatementIngest)(reqOptions) if desc, err = descForCommand(cmd); err != nil { - return + return 0, err } if stream, err = c.Client.DoPut(ctx, opts...); err != nil { - return + return 0, err } wr = flight.NewRecordWriter(stream, ipc.WithAllocator(c.Alloc), ipc.WithSchema(rdr.Schema())) @@ -285,20 +286,20 @@ func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqO for rdr.Next() { rec := rdr.Record() if err = wr.Write(rec); err != nil { - return + return 0, err } } if err = rdr.Err(); err != nil { - return + return 0, err } if err = stream.CloseSend(); err != nil { - return + return 0, err } if res, err = stream.Recv(); err != nil { - return + return 0, err } // Drain the stream, ensuring only a single result was sent by the server. @@ -308,7 +309,7 @@ func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqO } if err = proto.Unmarshal(res.GetAppMetadata(), &updateResult); err != nil { - return + return 0, err } return updateResult.GetRecordCount(), nil From 553c73348a81e8d2d04f1555d8a9d6e13ff4d73f Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky Date: Mon, 18 Mar 2024 15:42:33 -0400 Subject: [PATCH 07/13] handle server close on write stream --- go/arrow/flight/flightsql/client.go | 9 ++++++++- go/arrow/flight/record_batch_writer.go | 17 ++++++++++++++++- 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/go/arrow/flight/flightsql/client.go b/go/arrow/flight/flightsql/client.go index 315c5b82c10..cad5bfed1f0 100644 --- a/go/arrow/flight/flightsql/client.go +++ b/go/arrow/flight/flightsql/client.go @@ -285,7 +285,14 @@ func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqO for rdr.Next() { rec := rdr.Record() - if err = wr.Write(rec); err != nil { + err = wr.Write(rec) + if errors.Is(err, flight.ErrDataStreamClosed) { + // The stream was closed by the server, so we clean up and stop writing. + // The specific error will be retrieved in the server response. + rec.Release() + break + } + if err != nil { return 0, err } } diff --git a/go/arrow/flight/record_batch_writer.go b/go/arrow/flight/record_batch_writer.go index 6fbec8d7bd9..6d342d3bc6a 100644 --- a/go/arrow/flight/record_batch_writer.go +++ b/go/arrow/flight/record_batch_writer.go @@ -18,12 +18,18 @@ package flight import ( "bytes" + "errors" + "io" "github.com/apache/arrow/go/v16/arrow" "github.com/apache/arrow/go/v16/arrow/ipc" "github.com/apache/arrow/go/v16/arrow/memory" ) +var ( + ErrDataStreamClosed error = errors.New("the server unexpectedly closed the FlightData stream") +) + // DataStreamWriter is an interface that represents an Arrow Flight stream // writer that writes FlightData objects type DataStreamWriter interface { @@ -47,7 +53,16 @@ func (f *flightPayloadWriter) WritePayload(payload ipc.Payload) error { payload.SerializeBody(&f.buf) f.fd.DataBody = f.buf.Bytes() - return f.w.Send(&f.fd) + err := f.w.Send(&f.fd) + if err == io.EOF { + // gRPC returns io.EOF if the error was generated by the server. + // We return a more informative error to clients that hides this gRPC idiosyncrasy. + // The error can be retrieved subsequently by calling RecvMsg on the stream + // ref: https://pkg.go.dev/google.golang.org/grpc#ClientStream + return ErrDataStreamClosed + } + + return err } func (f *flightPayloadWriter) Close() error { return nil } From 5f2ed0320f82c1e5b07deaf9a10c6af09e1ceab0 Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky Date: Mon, 18 Mar 2024 17:52:52 -0400 Subject: [PATCH 08/13] add failure test and improve error handling --- go/arrow/flight/flightsql/client.go | 21 +++++++------ go/arrow/flight/flightsql/server.go | 18 ++++++++--- go/arrow/flight/flightsql/server_test.go | 40 ++++++++++++++++++++++-- 3 files changed, 62 insertions(+), 17 deletions(-) diff --git a/go/arrow/flight/flightsql/client.go b/go/arrow/flight/flightsql/client.go index cad5bfed1f0..d744384ca32 100644 --- a/go/arrow/flight/flightsql/client.go +++ b/go/arrow/flight/flightsql/client.go @@ -287,9 +287,8 @@ func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqO rec := rdr.Record() err = wr.Write(rec) if errors.Is(err, flight.ErrDataStreamClosed) { - // The stream was closed by the server, so we clean up and stop writing. + // The stream was closed by the server, so we stop writing. // The specific error will be retrieved in the server response. - rec.Release() break } if err != nil { @@ -309,17 +308,21 @@ func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqO return 0, err } - // Drain the stream, ensuring only a single result was sent by the server. - // Flight RPC allows multiple server results, so the check may be removed if Flight SQL ingestion defines semanatics for this scenario. - if _, err = stream.Recv(); err != io.EOF { - return 0, ErrTooManyPutResults - } - if err = proto.Unmarshal(res.GetAppMetadata(), &updateResult); err != nil { return 0, err } - return updateResult.GetRecordCount(), nil + // Drain the stream. If ingestion was successful, no more messages should arrive. + // If there was a failure, the next message contains the error and the DoPutUpdateResult + // we recieved indicates a partial ingestion if the RecordCount is non-zero. + for { + _, err := stream.Recv() + if err == io.EOF { + return updateResult.GetRecordCount(), nil + } else if err != nil { + return updateResult.GetRecordCount(), err + } + } } // GetCatalogs requests the list of catalogs from the server and diff --git a/go/arrow/flight/flightsql/server.go b/go/arrow/flight/flightsql/server.go index a72b79bccdc..cb732ae7566 100644 --- a/go/arrow/flight/flightsql/server.go +++ b/go/arrow/flight/flightsql/server.go @@ -1004,17 +1004,25 @@ func (f *flightSqlServer) DoPut(stream flight.FlightService_DoPutServer) error { } return stream.Send(out) case *pb.CommandStatementIngest: - recordCount, err := f.srv.DoPutCommandStatementIngest(stream.Context(), cmd, rdr) - if err != nil { - return err - } + // Even if there was an error, the server may have ingested some records. + // For this reason we send PutResult{recordCount} no matter what, potentially followed by an error + // if there was one. + recordCount, rpcErr := f.srv.DoPutCommandStatementIngest(stream.Context(), cmd, rdr) result := pb.DoPutUpdateResult{RecordCount: recordCount} out := &flight.PutResult{} if out.AppMetadata, err = proto.Marshal(&result); err != nil { return status.Errorf(codes.Internal, "failed to marshal PutResult: %s", err.Error()) } - return stream.Send(out) + + // If we fail to send the recordCount, just return an error outright + if err := stream.Send(out); err != nil { + return err + } + + // We successfully sent the recordCount. + // Send the error if one occurred in the RPC, otherwise this is nil. + return rpcErr default: return status.Error(codes.InvalidArgument, "the defined request is invalid") } diff --git a/go/arrow/flight/flightsql/server_test.go b/go/arrow/flight/flightsql/server_test.go index 414a46d18e9..d245c958033 100644 --- a/go/arrow/flight/flightsql/server_test.go +++ b/go/arrow/flight/flightsql/server_test.go @@ -179,12 +179,16 @@ func (*testServer) CloseSession(ctx context.Context, req *flight.CloseSessionReq } func (*testServer) DoPutCommandStatementIngest(ctx context.Context, cmd flightsql.StatementIngest, rdr flight.MessageReader) (int64, error) { - var nRecords int64 + var maxRows int64 = 50 + var nRows int64 for rdr.Next() { rec := rdr.Record() - nRecords += rec.NumRows() + if nRows+rec.NumRows() > maxRows { + return nRows, fmt.Errorf("ingested rows exceeded maximum of %d", maxRows) + } + nRows += rec.NumRows() } - return nRecords, nil + return nRows, nil } type FlightSqlServerSuite struct { @@ -338,6 +342,36 @@ func (s *FlightSqlServerSuite) TestExecuteIngest() { s.Equal(nRowsExpected, nRowsIngested) } +func (s *FlightSqlServerSuite) TestExecuteIngestWithServerError() { + nRecords := 11 // intentionally exceed maximum number of rows the server can ingest + nRowsPerRecord := 5 + reclist := generateRecords(s.cl.Alloc, nRecords, nRowsPerRecord) + for _, rec := range reclist { + defer rec.Release() + } + + rdr, err := array.NewRecordReader(reclist[0].Schema(), reclist) + s.NoError(err) + defer rdr.Release() + + nRowsIngested, err := s.cl.ExecuteIngest( + context.TODO(), + rdr, + &flightsql.ExecuteIngestOpts{ + TableDefinitionOptions: &flightsql.TableDefinitionOptions{ + IfNotExist: flightsql.TableDefinitionOptionsTableNotExistOptionCreate, + IfExists: flightsql.TableDefinitionOptionsTableExistsOptionReplace, + }, + Table: "test_table", + }, + ) + s.Error(err) + s.ErrorContains(err, "ingested rows exceeded maximum") + + nRowsExpected := int64(50) // max rows the server can ingest + s.Equal(nRowsExpected, nRowsIngested) +} + func generateRecords(alloc memory.Allocator, nRecords, nRowsPerRecord int) []arrow.Record { schema := arrow.NewSchema( []arrow.Field{ From 4665f36133cd3a0730af44fbc763ed68e56bbf6e Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky <33523178+joellubi@users.noreply.github.com> Date: Sat, 6 Apr 2024 09:09:47 -0400 Subject: [PATCH 09/13] Update docs/source/format/FlightSql.rst Co-authored-by: Andrew Lamb --- docs/source/format/FlightSql.rst | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/source/format/FlightSql.rst b/docs/source/format/FlightSql.rst index 0f99d77cb93..b27376e94e8 100644 --- a/docs/source/format/FlightSql.rst +++ b/docs/source/format/FlightSql.rst @@ -180,7 +180,8 @@ in the ``app_metadata`` field of the Flight RPC ``PutResult`` returned. Execute a bulk ingestion. When used with DoPut: load the stream of Arrow record batches into - the specified target table and return the number of rows ingested. + the specified target table and return the number of rows ingested + via a `DoPutUpdateResult` message. Flight Server Session Management -------------------------------- From 6787a4d0a17489c6a0dce7d27bbf3945fce28290 Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky Date: Tue, 9 Apr 2024 15:56:04 -0400 Subject: [PATCH 10/13] revert to return io.EOF instead of custom error --- go/arrow/flight/flightsql/client.go | 7 +++---- go/arrow/flight/record_batch_writer.go | 17 +---------------- 2 files changed, 4 insertions(+), 20 deletions(-) diff --git a/go/arrow/flight/flightsql/client.go b/go/arrow/flight/flightsql/client.go index d744384ca32..22050c1c0db 100644 --- a/go/arrow/flight/flightsql/client.go +++ b/go/arrow/flight/flightsql/client.go @@ -243,8 +243,6 @@ func (c *Client) ExecuteSubstraitUpdate(ctx context.Context, plan SubstraitPlan, return updateResult.GetRecordCount(), nil } -var ErrTooManyPutResults = fmt.Errorf("%w: server sent multiple PutResults, expected one ", arrow.ErrInvalid) - // ExecuteIngest is for executing a bulk ingestion and only returns the number of affected rows. // The provided RecordReader will be retained for the duration of the call, but it is the caller's // responsibility to release the original reference. @@ -286,9 +284,10 @@ func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqO for rdr.Next() { rec := rdr.Record() err = wr.Write(rec) - if errors.Is(err, flight.ErrDataStreamClosed) { - // The stream was closed by the server, so we stop writing. + if err == io.EOF { + // gRPC returns io.EOF if the error was generated by the server. // The specific error will be retrieved in the server response. + // ref: https://pkg.go.dev/google.golang.org/grpc#ClientStream break } if err != nil { diff --git a/go/arrow/flight/record_batch_writer.go b/go/arrow/flight/record_batch_writer.go index 6d342d3bc6a..6fbec8d7bd9 100644 --- a/go/arrow/flight/record_batch_writer.go +++ b/go/arrow/flight/record_batch_writer.go @@ -18,18 +18,12 @@ package flight import ( "bytes" - "errors" - "io" "github.com/apache/arrow/go/v16/arrow" "github.com/apache/arrow/go/v16/arrow/ipc" "github.com/apache/arrow/go/v16/arrow/memory" ) -var ( - ErrDataStreamClosed error = errors.New("the server unexpectedly closed the FlightData stream") -) - // DataStreamWriter is an interface that represents an Arrow Flight stream // writer that writes FlightData objects type DataStreamWriter interface { @@ -53,16 +47,7 @@ func (f *flightPayloadWriter) WritePayload(payload ipc.Payload) error { payload.SerializeBody(&f.buf) f.fd.DataBody = f.buf.Bytes() - err := f.w.Send(&f.fd) - if err == io.EOF { - // gRPC returns io.EOF if the error was generated by the server. - // We return a more informative error to clients that hides this gRPC idiosyncrasy. - // The error can be retrieved subsequently by calling RecvMsg on the stream - // ref: https://pkg.go.dev/google.golang.org/grpc#ClientStream - return ErrDataStreamClosed - } - - return err + return f.w.Send(&f.fd) } func (f *flightPayloadWriter) Close() error { return nil } From ca1e70a56f4a630e5b8842ec65ae39eae7a5385e Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky Date: Fri, 12 Apr 2024 13:49:10 -0400 Subject: [PATCH 11/13] improve client nil handling --- go/arrow/flight/flightsql/client.go | 10 ++++------ go/arrow/flight/flightsql/server_test.go | 7 +++++++ 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/go/arrow/flight/flightsql/client.go b/go/arrow/flight/flightsql/client.go index 22050c1c0db..fe42e65ccda 100644 --- a/go/arrow/flight/flightsql/client.go +++ b/go/arrow/flight/flightsql/client.go @@ -256,18 +256,16 @@ func (c *Client) ExecuteIngest(ctx context.Context, rdr array.RecordReader, reqO updateResult pb.DoPutUpdateResult ) + cmd := (*pb.CommandStatementIngest)(reqOptions) + // Servers cannot infer defaults for these parameters, so we validate the request to ensure they are set. - if reqOptions.TableDefinitionOptions == nil { + if cmd.GetTableDefinitionOptions() == nil { return 0, fmt.Errorf("cannot ExecuteIngest: invalid ExecuteIngestOpts, TableDefinitionOptions is required") } - if reqOptions.Table == "" { + if cmd.GetTable() == "" { return 0, fmt.Errorf("cannot ExecuteIngest: invalid ExecuteIngestOpts, Table is required") } - rdr.Retain() - defer rdr.Release() - - cmd := (*pb.CommandStatementIngest)(reqOptions) if desc, err = descForCommand(cmd); err != nil { return 0, err } diff --git a/go/arrow/flight/flightsql/server_test.go b/go/arrow/flight/flightsql/server_test.go index d245c958033..c9dca0be3bc 100644 --- a/go/arrow/flight/flightsql/server_test.go +++ b/go/arrow/flight/flightsql/server_test.go @@ -301,6 +301,13 @@ func (s *FlightSqlServerSuite) TestExecutePoll() { s.Len(poll.GetInfo().Endpoint, 2) } +func (s *FlightSqlServerSuite) TestExecuteIngestNil() { + // Ingest with nil options errors, but does not panic + nRecords, err := s.cl.ExecuteIngest(context.TODO(), nil, nil) + s.Error(err) + s.Equal(int64(0), nRecords) +} + func (s *FlightSqlServerSuite) TestExecuteIngestInvalid() { reclist := []arrow.Record{} rdr, err := array.NewRecordReader(arrow.NewSchema([]arrow.Field{}, nil), reclist) From b568efe2aa5d10c53f517a94ab2f6132f300b560 Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky Date: Fri, 12 Apr 2024 13:49:26 -0400 Subject: [PATCH 12/13] consolidate table def types into a single block --- go/arrow/flight/flightsql/types.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/go/arrow/flight/flightsql/types.go b/go/arrow/flight/flightsql/types.go index 265c430670c..36fac587ddb 100644 --- a/go/arrow/flight/flightsql/types.go +++ b/go/arrow/flight/flightsql/types.go @@ -881,10 +881,11 @@ const ( type CreatePreparedStatementResult = pb.ActionCreatePreparedStatementResult -type TableDefinitionOptions = pb.CommandStatementIngest_TableDefinitionOptions - -type TableDefinitionOptionsTableNotExistOption = pb.CommandStatementIngest_TableDefinitionOptions_TableNotExistOption -type TableDefinitionOptionsTableExistsOption = pb.CommandStatementIngest_TableDefinitionOptions_TableExistsOption +type ( + TableDefinitionOptions = pb.CommandStatementIngest_TableDefinitionOptions + TableDefinitionOptionsTableNotExistOption = pb.CommandStatementIngest_TableDefinitionOptions_TableNotExistOption + TableDefinitionOptionsTableExistsOption = pb.CommandStatementIngest_TableDefinitionOptions_TableExistsOption +) const ( TableDefinitionOptionsTableNotExistOptionUnspecified = pb.CommandStatementIngest_TableDefinitionOptions_TABLE_NOT_EXIST_OPTION_UNSPECIFIED From 60a9bd4f9cd43a4f74098a294fb532a897f4da77 Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky Date: Fri, 12 Apr 2024 13:50:00 -0400 Subject: [PATCH 13/13] getIngestOptions return pointer --- go/arrow/internal/flight_integration/scenario.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/go/arrow/internal/flight_integration/scenario.go b/go/arrow/internal/flight_integration/scenario.go index 504726e024e..56c396f49ed 100644 --- a/go/arrow/internal/flight_integration/scenario.go +++ b/go/arrow/internal/flight_integration/scenario.go @@ -2979,7 +2979,7 @@ func (m *flightSqlIngestionScenarioTester) RunClient(addr string, opts ...grpc.D func (m *flightSqlIngestionScenarioTester) ValidateIngestion(client *flightsql.Client) error { ctx := context.Background() opts := getIngestOptions() - ingestResult, err := client.ExecuteIngest(ctx, getIngestRecords(), &opts) + ingestResult, err := client.ExecuteIngest(ctx, getIngestRecords(), opts) if err != nil { return err } @@ -3039,7 +3039,7 @@ func (m *flightSqlIngestionScenarioTester) DoPutCommandStatementIngest(ctx conte } // Options to assert before/after mocked ingest call -func getIngestOptions() flightsql.ExecuteIngestOpts { +func getIngestOptions() *flightsql.ExecuteIngestOpts { tableDefinitionOptions := flightsql.TableDefinitionOptions{ IfNotExist: flightsql.TableDefinitionOptionsTableNotExistOptionCreate, IfExists: flightsql.TableDefinitionOptionsTableExistsOptionReplace, @@ -3054,7 +3054,7 @@ func getIngestOptions() flightsql.ExecuteIngestOpts { "key2": "val2", } - return flightsql.ExecuteIngestOpts{ + return &flightsql.ExecuteIngestOpts{ TableDefinitionOptions: &tableDefinitionOptions, Table: table, Schema: &schema,