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 92c088b7fae08..15318a8d7a465 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 5fa561cd7af28..665c1f1ba036a 100644 --- a/cpp/src/arrow/flight/integration_tests/test_integration.cc +++ b/cpp/src/arrow/flight/integration_tests/test_integration.cc @@ -1069,6 +1069,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"; @@ -2124,6 +2125,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) { @@ -2166,6 +2288,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 d0552e33df5c2..37b6a0b32e45d 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 9782611dbadcd..c37c640e653a4 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 a5cb842de8f49..cae3542b4faf8 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 df46004474a39..7b5d71678f3de 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 b41488b68f232..fe90c08ed2019 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 b50d5c5c2644e..5b66842b25926 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 5573c0040761f..1a43e4bdff306 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. @@ -185,6 +191,13 @@ 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 + via a `DoPutUpdateResult` message. + Flight Server Session Management -------------------------------- @@ -241,6 +254,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 0000000000000..781289d77b41a --- /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 0000000000000..e2aa72459afa5 --- /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 3282ee4f47304..bf3fcb6c3d229 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 09e4974ae9bfc..fe42e65ccda2e 100644 --- a/go/arrow/flight/flightsql/client.go +++ b/go/arrow/flight/flightsql/client.go @@ -243,6 +243,85 @@ func (c *Client) ExecuteSubstraitUpdate(ctx context.Context, plan SubstraitPlan, return updateResult.GetRecordCount(), nil } +// 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) (int64, error) { + var ( + err error + desc *flight.FlightDescriptor + stream pb.FlightService_DoPutClient + wr *flight.Writer + res *pb.PutResult + 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 cmd.GetTableDefinitionOptions() == nil { + return 0, fmt.Errorf("cannot ExecuteIngest: invalid ExecuteIngestOpts, TableDefinitionOptions is required") + } + if cmd.GetTable() == "" { + return 0, fmt.Errorf("cannot ExecuteIngest: invalid ExecuteIngestOpts, Table is required") + } + + if desc, err = descForCommand(cmd); err != nil { + return 0, err + } + + if stream, err = c.Client.DoPut(ctx, opts...); err != nil { + return 0, err + } + + wr = flight.NewRecordWriter(stream, ipc.WithAllocator(c.Alloc), ipc.WithSchema(rdr.Schema())) + defer wr.Close() + + wr.SetFlightDescriptor(desc) + + for rdr.Next() { + rec := rdr.Record() + err = wr.Write(rec) + 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 { + return 0, err + } + } + + if err = rdr.Err(); err != nil { + return 0, err + } + + if err = stream.CloseSend(); err != nil { + return 0, err + } + + if res, err = stream.Recv(); err != nil { + return 0, err + } + + if err = proto.Unmarshal(res.GetAppMetadata(), &updateResult); err != nil { + return 0, err + } + + // 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 // 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 7bc15ab4295f1..cb732ae75666f 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() } @@ -985,6 +1003,26 @@ func (f *flightSqlServer) DoPut(stream flight.FlightService_DoPutServer) error { return status.Errorf(codes.Internal, "failed to marshal PutResult: %s", err.Error()) } return stream.Send(out) + case *pb.CommandStatementIngest: + // 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()) + } + + // 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 df619e7a24140..c9dca0be3bcbc 100644 --- a/go/arrow/flight/flightsql/server_test.go +++ b/go/arrow/flight/flightsql/server_test.go @@ -178,6 +178,19 @@ 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 maxRows int64 = 50 + var nRows int64 + for rdr.Next() { + rec := rdr.Record() + if nRows+rec.NumRows() > maxRows { + return nRows, fmt.Errorf("ingested rows exceeded maximum of %d", maxRows) + } + nRows += rec.NumRows() + } + return nRows, nil +} + type FlightSqlServerSuite struct { suite.Suite @@ -202,9 +215,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 } @@ -281,6 +301,111 @@ 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) + s.NoError(err) + defer rdr.Release() + + // Cannot execute ingest without specifying required options + nRecords, err := s.cl.ExecuteIngest(context.TODO(), rdr, &flightsql.ExecuteIngestOpts{}) + s.Error(err) + s.Equal(int64(0), nRecords) +} + +func (s *FlightSqlServerSuite) TestExecuteIngest() { + 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() + + nRowsIngested, err := s.cl.ExecuteIngest( + context.TODO(), + rdr, + &flightsql.ExecuteIngestOpts{ + TableDefinitionOptions: &flightsql.TableDefinitionOptions{ + IfNotExist: flightsql.TableDefinitionOptionsTableNotExistOptionCreate, + IfExists: flightsql.TableDefinitionOptionsTableExistsOptionReplace, + }, + Table: "test_table", + }, + ) + s.NoError(err) + + nRowsExpected := int64(nRecords * nRowsPerRecord) + 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{ + {Name: "one", Type: arrow.FixedWidthTypes.Boolean}, + {Name: "two", Type: arrow.BinaryTypes.String}, + {Name: "three", Type: arrow.PrimitiveTypes.Int64}, + }, + nil, + ) + + 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 { suite.Suite @@ -459,6 +584,36 @@ func (s *UnimplementedFlightSqlServerSuite) TestDoGet() { } } +func (s *UnimplementedFlightSqlServerSuite) TestExecuteIngest() { + 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, + &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()) + s.Equal("DoPutCommandStatementIngest not implemented", st.Message()) + 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 c70a8bdc4ec26..36fac587ddb27 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,20 @@ const ( ) type CreatePreparedStatementResult = pb.ActionCreatePreparedStatementResult + +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 + 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/Flight.pb.go b/go/arrow/flight/gen/flight/Flight.pb.go index 42d4493c8f3e6..d9477ee062fa8 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 d886bc6bdb70b..3ff7c5fc147a6 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. @@ -592,6 +603,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 +696,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 +2187,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 +2342,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 +2355,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 @@ -4184,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 @@ -4200,7 +4433,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 +4446,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 +4459,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 +4501,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 +4514,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 +4527,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 +4556,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 +4569,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 +4582,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 +4592,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), @@ -4606,513 +4895,586 @@ 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, 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, + 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, - 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, + 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 +5489,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 +5903,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 +5915,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 +5927,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 +5950,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 +5975,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 237cb1fe2dfb5..11bbb00131ddb 100644 --- a/go/arrow/flight/gen/flight/Flight_grpc.pb.go +++ b/go/arrow/flight/gen/flight/Flight_grpc.pb.go @@ -1,7 +1,7 @@ // 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 diff --git a/go/arrow/internal/flight_integration/scenario.go b/go/arrow/internal/flight_integration/scenario.go index edddd201ddaea..56c396f49edc1 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 +}