From 4fcdd01c1076343a8617027b3917c01330955b9c Mon Sep 17 00:00:00 2001 From: Joel Lubinitsky <33523178+joellubi@users.noreply.github.com> Date: Wed, 17 Apr 2024 11:21:49 -0400 Subject: [PATCH] GH-38255: [Go][C++] Implement Flight SQL Bulk Ingestion (#38385) ### Rationale for this change It was suggested in the discussion around https://github.com/apache/arrow-adbc/issues/1107 for the Flight SQL ADBC driver that an "Ingest" command would be a helpful addition to the Flight SQL specification. This command would enable a Flight SQL client to provide a FlightData stream to the server without needing to know its SQL syntax, and have that stream loaded into a target table by whichever means the server deems appropriate. ### What changes are included in this PR? - Format: - Add CommandStatementIngest message type to Flight SQL proto definition - Add FLIGHT_SQL_SERVER_BULK_INGESTION and FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED options for SqlInfo - Go: - Generate pb - Server-side implementation - Client-side implementation - Unit + integration tests - C++: - Server-side implementation - Client-side implementation - Integration tests ### Are these changes tested? Yes, see `server_test.go`, `scenario.go`, and `test_integration.cc`. ### Are there any user-facing changes? Yes, new Flight SQL client and server functionality is being added. Changes are not expected to break existing users. * Closes: #38255 Lead-authored-by: Joel Lubinitsky Co-authored-by: Joel Lubinitsky <33523178+joellubi@users.noreply.github.com> Co-authored-by: Antoine Pitrou Co-authored-by: Andrew Lamb Signed-off-by: Matt Topol --- arrow/flight/flightsql/client.go | 79 + arrow/flight/flightsql/server.go | 38 + arrow/flight/flightsql/server_test.go | 155 ++ arrow/flight/flightsql/types.go | 43 + arrow/flight/gen/flight/Flight.pb.go | 16 +- arrow/flight/gen/flight/FlightSql.pb.go | 1560 +++++++++++------ arrow/flight/gen/flight/Flight_grpc.pb.go | 2 +- arrow/internal/flight_integration/scenario.go | 136 ++ 8 files changed, 1440 insertions(+), 589 deletions(-) diff --git a/arrow/flight/flightsql/client.go b/arrow/flight/flightsql/client.go index 09e4974a..fe42e65c 100644 --- a/arrow/flight/flightsql/client.go +++ b/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/arrow/flight/flightsql/server.go b/arrow/flight/flightsql/server.go index 7bc15ab4..cb732ae7 100644 --- a/arrow/flight/flightsql/server.go +++ b/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/arrow/flight/flightsql/server_test.go b/arrow/flight/flightsql/server_test.go index df619e7a..c9dca0be 100644 --- a/arrow/flight/flightsql/server_test.go +++ b/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/arrow/flight/flightsql/types.go b/arrow/flight/flightsql/types.go index c70a8bdc..36fac587 100644 --- a/arrow/flight/flightsql/types.go +++ b/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/arrow/flight/gen/flight/Flight.pb.go b/arrow/flight/gen/flight/Flight.pb.go index 42d4493c..d9477ee0 100644 --- a/arrow/flight/gen/flight/Flight.pb.go +++ b/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/arrow/flight/gen/flight/FlightSql.pb.go b/arrow/flight/gen/flight/FlightSql.pb.go index d886bc6b..3ff7c5fc 100644 --- a/arrow/flight/gen/flight/FlightSql.pb.go +++ b/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/arrow/flight/gen/flight/Flight_grpc.pb.go b/arrow/flight/gen/flight/Flight_grpc.pb.go index 237cb1fe..11bbb001 100644 --- a/arrow/flight/gen/flight/Flight_grpc.pb.go +++ b/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/arrow/internal/flight_integration/scenario.go b/arrow/internal/flight_integration/scenario.go index edddd201..56c396f4 100644 --- a/arrow/internal/flight_integration/scenario.go +++ b/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 +}