From e8ed4c2fe2f7d19cbb8b622274280b344ed4797b Mon Sep 17 00:00:00 2001 From: Matt Topol Date: Thu, 11 Aug 2022 16:32:53 -0400 Subject: [PATCH] ARROW-17326: [Go][FlightSQL] Add FlightSQL support for Go (#13828) This implements initial support for FlightSQL in Golang and is sufficient to pass the flight_sql integration scenario. As a follow-up in a subsequent PR i'll implement an equivalent example to the C++ SQLite example using Go. Also closes #12496 Authored-by: Matt Topol Signed-off-by: Matt Topol --- dev/archery/archery/integration/runner.py | 2 +- format/Flight.proto | 2 +- format/FlightSql.proto | 1 + go/arrow/array/record.go | 4 + go/arrow/array/record_test.go | 3 +- go/arrow/array/union.go | 11 + go/arrow/flight/flightsql/client.go | 483 ++ go/arrow/flight/flightsql/client_test.go | 469 ++ go/arrow/flight/flightsql/column_metadata.go | 212 + .../flightsql/schema_ref/reference_schemas.go | 106 + go/arrow/flight/flightsql/server.go | 718 +++ go/arrow/flight/flightsql/server_test.go | 212 + go/arrow/flight/flightsql/sql_info.go | 93 + go/arrow/flight/flightsql/types.go | 745 +++ go/arrow/flight/gen.go | 1 + go/arrow/flight/internal/flight/Flight.pb.go | 46 +- .../flight/internal/flight/FlightSql.pb.go | 4396 +++++++++++++++++ go/arrow/flight/record_batch_reader.go | 83 +- go/arrow/flight/record_batch_writer.go | 10 + .../internal/flight_integration/scenario.go | 518 ++ go/arrow/ipc/cmd/arrow-cat/main.go | 2 +- go/arrow/ipc/ipc.go | 10 + go/arrow/ipc/reader.go | 50 +- go/arrow/math/float64_amd64.go | 1 + go/arrow/math/float64_avx2_amd64.go | 1 + go/arrow/math/float64_sse4_amd64.go | 1 + go/arrow/math/int64_amd64.go | 1 + go/arrow/math/int64_avx2_amd64.go | 1 + go/arrow/math/int64_sse4_amd64.go | 1 + go/arrow/math/uint64_amd64.go | 1 + go/arrow/math/uint64_avx2_amd64.go | 1 + go/arrow/math/uint64_sse4_amd64.go | 1 + go/arrow/scalar/numeric.gen.go | 2 +- go/go.mod | 1 + 34 files changed, 8154 insertions(+), 35 deletions(-) create mode 100644 go/arrow/flight/flightsql/client.go create mode 100644 go/arrow/flight/flightsql/client_test.go create mode 100644 go/arrow/flight/flightsql/column_metadata.go create mode 100644 go/arrow/flight/flightsql/schema_ref/reference_schemas.go create mode 100644 go/arrow/flight/flightsql/server.go create mode 100644 go/arrow/flight/flightsql/server_test.go create mode 100644 go/arrow/flight/flightsql/sql_info.go create mode 100644 go/arrow/flight/flightsql/types.go create mode 100644 go/arrow/flight/internal/flight/FlightSql.pb.go diff --git a/dev/archery/archery/integration/runner.py b/dev/archery/archery/integration/runner.py index 6d6adb3c29cd3..478cb73da07cb 100644 --- a/dev/archery/archery/integration/runner.py +++ b/dev/archery/archery/integration/runner.py @@ -435,7 +435,7 @@ def run_all_tests(with_cpp=True, with_java=True, with_js=True, Scenario( "flight_sql", description="Ensure Flight SQL protocol is working as expected.", - skip={"Rust", "Go"} + skip={"Rust"} ), ] diff --git a/format/Flight.proto b/format/Flight.proto index 87e5fda796d53..635b1793d2bab 100644 --- a/format/Flight.proto +++ b/format/Flight.proto @@ -19,7 +19,7 @@ syntax = "proto3"; option java_package = "org.apache.arrow.flight.impl"; -option go_package = "github.com/apache/arrow/go/flight;flight"; +option go_package = "github.com/apache/arrow/go/arrow/flight/internal/flight"; option csharp_namespace = "Apache.Arrow.Flight.Protocol"; package arrow.flight.protocol; diff --git a/format/FlightSql.proto b/format/FlightSql.proto index 149f8831e139f..859427b68804b 100644 --- a/format/FlightSql.proto +++ b/format/FlightSql.proto @@ -20,6 +20,7 @@ syntax = "proto3"; import "google/protobuf/descriptor.proto"; option java_package = "org.apache.arrow.flight.sql.impl"; +option go_package = "github.com/apache/arrow/go/arrow/flight/internal/flight"; package arrow.flight.protocol.sql; /* diff --git a/go/arrow/array/record.go b/go/arrow/array/record.go index 7890b0e99b938..1c122c28f028b 100644 --- a/go/arrow/array/record.go +++ b/go/arrow/array/record.go @@ -153,6 +153,10 @@ func NewRecord(schema *arrow.Schema, cols []arrow.Array, nrows int64) *simpleRec } func (rec *simpleRecord) validate() error { + if rec.rows == 0 && len(rec.arrs) == 0 { + return nil + } + if len(rec.arrs) != len(rec.schema.Fields()) { return fmt.Errorf("arrow/array: number of columns/fields mismatch") } diff --git a/go/arrow/array/record_test.go b/go/arrow/array/record_test.go index d0cfd68c52384..d5fbeb8c8e6dc 100644 --- a/go/arrow/array/record_test.go +++ b/go/arrow/array/record_test.go @@ -135,8 +135,7 @@ func TestRecord(t *testing.T) { { schema: schema, cols: nil, - rows: -1, - err: fmt.Errorf("arrow/array: number of columns/fields mismatch"), + rows: 0, }, { schema: schema, diff --git a/go/arrow/array/union.go b/go/arrow/array/union.go index a7414fef4dd04..da15f63b336d7 100644 --- a/go/arrow/array/union.go +++ b/go/arrow/array/union.go @@ -683,6 +683,10 @@ type UnionBuilder interface { // Mode returns what kind of Union is being built, either arrow.SparseMode // or arrow.DenseMode Mode() arrow.UnionMode + // Child returns the builder for the requested child index. + // If an invalid index is requested (e.g. <0 or >len(children)) + // then this will panic. + Child(idx int) Builder } type unionBuilder struct { @@ -734,6 +738,13 @@ func newUnionBuilder(mem memory.Allocator, children []Builder, typ arrow.UnionTy return b } +func (b *unionBuilder) Child(idx int) Builder { + if idx < 0 || idx > len(b.children) { + panic("arrow/array: invalid child index for union builder") + } + return b.children[idx] +} + func (b *unionBuilder) Mode() arrow.UnionMode { return b.mode } func (b *unionBuilder) reserve(elements int, resize func(int)) { diff --git a/go/arrow/flight/flightsql/client.go b/go/arrow/flight/flightsql/client.go new file mode 100644 index 0000000000000..2f57d05484f76 --- /dev/null +++ b/go/arrow/flight/flightsql/client.go @@ -0,0 +1,483 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package flightsql + +import ( + "context" + "errors" + "io" + + "github.com/apache/arrow/go/v10/arrow" + "github.com/apache/arrow/go/v10/arrow/array" + "github.com/apache/arrow/go/v10/arrow/flight" + pb "github.com/apache/arrow/go/v10/arrow/flight/internal/flight" + "github.com/apache/arrow/go/v10/arrow/ipc" + "github.com/apache/arrow/go/v10/arrow/memory" + "google.golang.org/grpc" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/anypb" +) + +// NewClient is a convenience function to automatically construct +// a flight.Client and return a flightsql.Client containing it rather +// than having to manually construct both yourself. It just delegates +// its arguments to flight.NewClientWithMiddleware to create the +// underlying Flight Client. +func NewClient(addr string, auth flight.ClientAuthHandler, middleware []flight.ClientMiddleware, opts ...grpc.DialOption) (*Client, error) { + cl, err := flight.NewClientWithMiddleware(addr, auth, middleware, opts...) + if err != nil { + return nil, err + } + return &Client{cl}, nil +} + +// Client wraps a regular Flight RPC Client to provide the FlightSQL +// interface functions and methods. +type Client struct { + Client flight.Client +} + +func descForCommand(cmd proto.Message) (*flight.FlightDescriptor, error) { + var any anypb.Any + if err := any.MarshalFrom(cmd); err != nil { + return nil, err + } + + data, err := proto.Marshal(&any) + if err != nil { + return nil, err + } + return &flight.FlightDescriptor{ + Type: flight.DescriptorCMD, + Cmd: data, + }, nil +} + +func flightInfoForCommand(ctx context.Context, cl *Client, cmd proto.Message, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + desc, err := descForCommand(cmd) + if err != nil { + return nil, err + } + return cl.getFlightInfo(ctx, desc, opts...) +} + +// Execute executes the desired query on the server and returns a FlightInfo +// object describing where to retrieve the results. +func (c *Client) Execute(ctx context.Context, query string, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + cmd := pb.CommandStatementQuery{Query: query} + return flightInfoForCommand(ctx, c, &cmd, opts...) +} + +// ExecuteUpdate is for executing an update query and only returns the number of affected rows. +func (c *Client) ExecuteUpdate(ctx context.Context, query string, opts ...grpc.CallOption) (n int64, err error) { + var ( + cmd pb.CommandStatementUpdate + desc *flight.FlightDescriptor + stream pb.FlightService_DoPutClient + res *pb.PutResult + updateResult pb.DoPutUpdateResult + ) + + cmd.Query = query + if desc, err = descForCommand(&cmd); err != nil { + return + } + + if stream, err = c.Client.DoPut(ctx, opts...); err != nil { + return + } + + if err = stream.Send(&flight.FlightData{FlightDescriptor: desc}); err != nil { + return + } + + if err = stream.CloseSend(); err != nil { + return + } + + if res, err = stream.Recv(); err != nil { + return + } + + if err = proto.Unmarshal(res.GetAppMetadata(), &updateResult); err != nil { + return + } + + return updateResult.GetRecordCount(), nil +} + +// GetCatalogs requests the list of catalogs from the server and +// returns a flightInfo object where the response can be retrieved +func (c *Client) GetCatalogs(ctx context.Context, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + return flightInfoForCommand(ctx, c, &pb.CommandGetCatalogs{}, opts...) +} + +// GetDBSchemas requests the list of schemas from the database and +// returns a FlightInfo object where the response can be retrieved +func (c *Client) GetDBSchemas(ctx context.Context, cmdOpts *GetDBSchemasOpts, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + return flightInfoForCommand(ctx, c, (*pb.CommandGetDbSchemas)(cmdOpts), opts...) +} + +// DoGet uses the provided flight ticket to request the stream of data. +// It returns a recordbatch reader to stream the results. Release +// should be called on the reader when done. +func (c *Client) DoGet(ctx context.Context, in *flight.Ticket, opts ...grpc.CallOption) (*flight.Reader, error) { + stream, err := c.Client.DoGet(ctx, in, opts...) + if err != nil { + return nil, err + } + + return flight.NewRecordReader(stream) +} + +// GetTables requests a list of tables from the server, with the provided +// options describing how to make the request (filter patterns, if the schema +// should be returned, etc.). Returns a FlightInfo object where the response +// can be retrieved. +func (c *Client) GetTables(ctx context.Context, reqOptions *GetTablesOpts, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + return flightInfoForCommand(ctx, c, (*pb.CommandGetTables)(reqOptions), opts...) +} + +// GetPrimaryKeys requests the primary keys for a specific table from the +// server, specified using a TableRef. Returns a FlightInfo object where +// the response can be retrieved. +func (c *Client) GetPrimaryKeys(ctx context.Context, ref TableRef, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + cmd := pb.CommandGetPrimaryKeys{ + Catalog: ref.Catalog, + DbSchema: ref.DBSchema, + Table: ref.Table, + } + return flightInfoForCommand(ctx, c, &cmd, opts...) +} + +// GetExportedKeys retrieves a description about the foreign key columns +// that reference the primary key columns of the specified table. Returns +// a FlightInfo object where the response can be retrieved. +func (c *Client) GetExportedKeys(ctx context.Context, ref TableRef, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + cmd := pb.CommandGetExportedKeys{ + Catalog: ref.Catalog, + DbSchema: ref.DBSchema, + Table: ref.Table, + } + return flightInfoForCommand(ctx, c, &cmd, opts...) +} + +// GetImportedKeys returns the foreign key columns for the specified table. +// Returns a FlightInfo object indicating where the response can be retrieved. +func (c *Client) GetImportedKeys(ctx context.Context, ref TableRef, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + cmd := pb.CommandGetImportedKeys{ + Catalog: ref.Catalog, + DbSchema: ref.DBSchema, + Table: ref.Table, + } + return flightInfoForCommand(ctx, c, &cmd, opts...) +} + +// GetCrossReference retrieves a description of the foreign key columns +// in the specified ForeignKey table that reference the primary key or +// columns representing a restraint of the parent table (could be the same +// or a different table). Returns a FlightInfo object indicating where +// the response can be retrieved with DoGet. +func (c *Client) GetCrossReference(ctx context.Context, pkTable, fkTable TableRef, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + cmd := pb.CommandGetCrossReference{ + PkCatalog: pkTable.Catalog, + PkDbSchema: pkTable.DBSchema, + PkTable: pkTable.Table, + FkCatalog: fkTable.Catalog, + FkDbSchema: fkTable.DBSchema, + FkTable: fkTable.Table, + } + return flightInfoForCommand(ctx, c, &cmd, opts...) +} + +// GetTableTypes requests a list of the types of tables available on this +// server. Returns a FlightInfo object indicating where the response can +// be retrieved. +func (c *Client) GetTableTypes(ctx context.Context, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + return flightInfoForCommand(ctx, c, &pb.CommandGetTableTypes{}, opts...) +} + +// GetXdbcTypeInfo requests the information about all the data types supported +// (dataType == nil) or a specific data type. Returns a FlightInfo object +// indicating where the response can be retrieved. +func (c *Client) GetXdbcTypeInfo(ctx context.Context, dataType *int32, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + return flightInfoForCommand(ctx, c, &pb.CommandGetXdbcTypeInfo{DataType: dataType}, opts...) +} + +// GetSqlInfo returns a list of the requested SQL information corresponding +// to the values in the info slice. Returns a FlightInfo object indicating +// where the response can be retrieved. +func (c *Client) GetSqlInfo(ctx context.Context, info []SqlInfo, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + cmd := &pb.CommandGetSqlInfo{Info: make([]uint32, len(info))} + + for i, v := range info { + cmd.Info[i] = uint32(v) + } + return flightInfoForCommand(ctx, c, cmd, opts...) +} + +// Prepare creates a PreparedStatement object for the specified query. +// The resulting PreparedStatement object should be Closed when no longer +// needed. It will maintain a reference to this Client for use to execute +// and use the specified allocator for any allocations it needs to perform. +func (c *Client) Prepare(ctx context.Context, mem memory.Allocator, query string, opts ...grpc.CallOption) (prep *PreparedStatement, err error) { + const actionType = CreatePreparedStatementActionType + var ( + cmd, cmdResult anypb.Any + res *pb.Result + request pb.ActionCreatePreparedStatementRequest + result pb.ActionCreatePreparedStatementResult + action pb.Action + stream pb.FlightService_DoActionClient + dsSchema, paramSchema *arrow.Schema + ) + + request.Query = query + if err = cmd.MarshalFrom(&request); err != nil { + return + } + + action.Type = actionType + if action.Body, err = proto.Marshal(&cmd); err != nil { + return + } + + if stream, err = c.Client.DoAction(ctx, &action, opts...); err != nil { + return + } + + if res, err = stream.Recv(); err != nil { + return + } + + if err = proto.Unmarshal(res.Body, &cmdResult); err != nil { + return + } + + if err = cmdResult.UnmarshalTo(&result); err != nil { + return + } + + if result.DatasetSchema != nil { + dsSchema, err = flight.DeserializeSchema(result.DatasetSchema, mem) + if err != nil { + return + } + } + if result.ParameterSchema != nil { + paramSchema, err = flight.DeserializeSchema(result.ParameterSchema, mem) + if err != nil { + return + } + } + + prep = &PreparedStatement{ + client: c, + opts: opts, + handle: result.PreparedStatementHandle, + datasetSchema: dsSchema, + paramSchema: paramSchema, + } + return +} + +func (c *Client) getFlightInfo(ctx context.Context, desc *flight.FlightDescriptor, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + return c.Client.GetFlightInfo(ctx, desc, opts...) +} + +// Close will close the underlying flight Client in use by this flightsql.Client +func (c *Client) Close() error { return c.Client.Close() } + +// PreparedStatement represents a constructed PreparedStatement on the server +// and maintains a reference to the Client that created it along with the +// prepared statement handle. +// +// If the server returned the Dataset Schema or Parameter Binding schemas +// at creation, they will also be accessible from this object. Close +// should be called when no longer needed. +type PreparedStatement struct { + client *Client + opts []grpc.CallOption + handle []byte + datasetSchema *arrow.Schema + paramSchema *arrow.Schema + paramBinding arrow.Record + closed bool +} + +// Execute executes the prepared statement on the server and returns a FlightInfo +// indicating where to retrieve the response. If SetParameters has been called +// then the parameter bindings will be sent before execution. +// +// Will error if already closed. +func (p *PreparedStatement) Execute(ctx context.Context) (*flight.FlightInfo, error) { + if p.closed { + return nil, errors.New("arrow/flightsql: prepared statement already closed") + } + + cmd := &pb.CommandPreparedStatementQuery{PreparedStatementHandle: p.handle} + + desc, err := descForCommand(cmd) + if err != nil { + return nil, err + } + + if p.paramBinding != nil && p.paramBinding.NumRows() > 0 { + pstream, err := p.client.Client.DoPut(ctx, p.opts...) + if err != nil { + return nil, err + } + + wr := flight.NewRecordWriter(pstream, ipc.WithSchema(p.paramBinding.Schema())) + wr.SetFlightDescriptor(desc) + if err = wr.Write(p.paramBinding); err != nil { + return nil, err + } + if err = wr.Close(); err != nil { + return nil, err + } + pstream.CloseSend() + + // wait for the server to ack the result + if _, err = pstream.Recv(); err != nil && err != io.EOF { + return nil, err + } + } + + return p.client.getFlightInfo(ctx, desc, p.opts...) +} + +// ExecuteUpdate executes the prepared statement update query on the server +// and returns the number of rows affected. If SetParameters was called, +// the parameter bindings will be sent with the request to execute. +func (p *PreparedStatement) ExecuteUpdate(ctx context.Context) (nrecords int64, err error) { + if p.closed { + return 0, errors.New("arrow/flightsql: prepared statement already closed") + } + + var ( + execCmd = &pb.CommandPreparedStatementUpdate{PreparedStatementHandle: p.handle} + desc *flight.FlightDescriptor + pstream pb.FlightService_DoPutClient + wr *flight.Writer + res *pb.PutResult + updateResult pb.DoPutUpdateResult + ) + + desc, err = descForCommand(execCmd) + if err != nil { + return + } + + if pstream, err = p.client.Client.DoPut(ctx, p.opts...); err != nil { + return + } + if p.paramBinding != nil && p.paramBinding.NumRows() > 0 { + wr = flight.NewRecordWriter(pstream, ipc.WithSchema(p.paramBinding.Schema())) + wr.SetFlightDescriptor(desc) + if err = wr.Write(p.paramBinding); err != nil { + return + } + } else { + schema := arrow.NewSchema([]arrow.Field{}, nil) + wr = flight.NewRecordWriter(pstream, ipc.WithSchema(schema)) + wr.SetFlightDescriptor(desc) + rec := array.NewRecord(schema, []arrow.Array{}, 0) + if err = wr.Write(rec); err != nil { + return + } + } + + if err = wr.Close(); err != nil { + return + } + if err = pstream.CloseSend(); err != nil { + return + } + if res, err = pstream.Recv(); err != nil { + return + } + + if err = proto.Unmarshal(res.GetAppMetadata(), &updateResult); err != nil { + return + } + + return updateResult.GetRecordCount(), nil +} + +// DatasetSchema may be nil if the server did not return it when creating the +// Prepared Statement. +func (p *PreparedStatement) DatasetSchema() *arrow.Schema { return p.datasetSchema } + +// ParameterSchema may be nil if the server did not return it when creating +// the prepared statement. +func (p *PreparedStatement) ParameterSchema() *arrow.Schema { return p.paramSchema } + +// SetParameters takes a record batch to send as the parameter bindings when +// executing. It should match the schema from ParameterSchema. +// +// This will call Retain on the record to ensure it doesn't get released +// out from under the statement. Release will be called on a previous +// binding record if it existed, and will be called upon calling Close +// on the PreparedStatement. +func (p *PreparedStatement) SetParameters(binding arrow.Record) { + if p.paramBinding != nil { + p.paramBinding.Release() + p.paramBinding = nil + } + p.paramBinding = binding + p.paramBinding.Retain() +} + +// Close calls release on any parameter binding record and sends +// a ClosePreparedStatement action to the server. After calling +// Close, the PreparedStatement should not be used again. +func (p *PreparedStatement) Close(ctx context.Context) error { + if p.closed { + return errors.New("arrow/flightsql: already closed") + } + + if p.paramBinding != nil { + p.paramBinding.Release() + p.paramBinding = nil + } + + const actionType = ClosePreparedStatementActionType + var ( + cmd anypb.Any + request pb.ActionClosePreparedStatementRequest + ) + + request.PreparedStatementHandle = p.handle + if err := cmd.MarshalFrom(&request); err != nil { + return err + } + + body, err := proto.Marshal(&cmd) + if err != nil { + return err + } + + action := &flight.Action{Type: actionType, Body: body} + _, err = p.client.Client.DoAction(ctx, action, p.opts...) + if err != nil { + return err + } + + p.closed = true + return nil +} diff --git a/go/arrow/flight/flightsql/client_test.go b/go/arrow/flight/flightsql/client_test.go new file mode 100644 index 0000000000000..92e468313910c --- /dev/null +++ b/go/arrow/flight/flightsql/client_test.go @@ -0,0 +1,469 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package flightsql_test + +import ( + "context" + "strings" + "testing" + + "github.com/apache/arrow/go/v10/arrow" + "github.com/apache/arrow/go/v10/arrow/array" + "github.com/apache/arrow/go/v10/arrow/flight" + "github.com/apache/arrow/go/v10/arrow/flight/flightsql" + pb "github.com/apache/arrow/go/v10/arrow/flight/internal/flight" + "github.com/apache/arrow/go/v10/arrow/memory" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/suite" + "google.golang.org/grpc" + "google.golang.org/grpc/metadata" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/anypb" +) + +type mockGrpcClientStream struct { + mock.Mock +} + +func (m *mockGrpcClientStream) Header() (metadata.MD, error) { panic("unimplemented") } +func (m *mockGrpcClientStream) Trailer() metadata.MD { panic("unimplemented") } +func (m *mockGrpcClientStream) CloseSend() error { return m.Called().Error(0) } +func (m *mockGrpcClientStream) Context() context.Context { return context.TODO() } +func (m *mockGrpcClientStream) SendMsg(msg interface{}) error { return m.Called(msg).Error(0) } +func (m *mockGrpcClientStream) RecvMsg(msg interface{}) error { return m.Called(msg).Error(0) } + +type FlightServiceClientMock struct { + mock.Mock +} + +func (m *FlightServiceClientMock) Authenticate(_ context.Context, opts ...grpc.CallOption) error { + return m.Called(opts).Error(0) +} + +func (m *FlightServiceClientMock) AuthenticateBasicToken(_ context.Context, user, pass string, opts ...grpc.CallOption) (context.Context, error) { + args := m.Called(user, pass, opts) + return args.Get(0).(context.Context), args.Error(1) +} + +func (m *FlightServiceClientMock) Close() error { + return m.Called().Error(0) +} + +func (m *FlightServiceClientMock) Handshake(ctx context.Context, opts ...grpc.CallOption) (flight.FlightService_HandshakeClient, error) { + panic("not implemented") // TODO: Implement +} + +func (m *FlightServiceClientMock) ListFlights(ctx context.Context, in *flight.Criteria, opts ...grpc.CallOption) (flight.FlightService_ListFlightsClient, error) { + panic("not implemented") // TODO: Implement +} + +func (m *FlightServiceClientMock) GetFlightInfo(ctx context.Context, in *flight.FlightDescriptor, opts ...grpc.CallOption) (*flight.FlightInfo, error) { + args := m.Called(in.Type, in.Cmd, opts) + return args.Get(0).(*flight.FlightInfo), args.Error(1) +} + +func (m *FlightServiceClientMock) GetSchema(ctx context.Context, in *flight.FlightDescriptor, opts ...grpc.CallOption) (*flight.SchemaResult, error) { + panic("not implemented") // TODO: Implement +} + +func (m *FlightServiceClientMock) DoGet(ctx context.Context, in *flight.Ticket, opts ...grpc.CallOption) (flight.FlightService_DoGetClient, error) { + panic("not implemented") // TODO: Implement +} + +func (m *FlightServiceClientMock) DoPut(ctx context.Context, opts ...grpc.CallOption) (flight.FlightService_DoPutClient, error) { + args := m.Called(opts) + return args.Get(0).(flight.FlightService_DoPutClient), args.Error(1) +} + +func (m *FlightServiceClientMock) DoExchange(ctx context.Context, opts ...grpc.CallOption) (flight.FlightService_DoExchangeClient, error) { + panic("not implemented") // TODO: Implement +} + +func (m *FlightServiceClientMock) DoAction(ctx context.Context, in *flight.Action, opts ...grpc.CallOption) (flight.FlightService_DoActionClient, error) { + args := m.Called(in.Type, in.Body, opts) + return args.Get(0).(flight.FlightService_DoActionClient), args.Error(1) +} + +func (m *FlightServiceClientMock) ListActions(ctx context.Context, in *flight.Empty, opts ...grpc.CallOption) (flight.FlightService_ListActionsClient, error) { + panic("not implemented") // TODO: Implement +} + +type FlightSqlClientSuite struct { + suite.Suite + + mockClient FlightServiceClientMock + callOpts []grpc.CallOption + sqlClient flightsql.Client +} + +func getDesc(cmd proto.Message) *flight.FlightDescriptor { + var anycmd anypb.Any + anycmd.MarshalFrom(cmd) + + data, _ := proto.Marshal(&anycmd) + return &flight.FlightDescriptor{ + Type: flight.DescriptorCMD, + Cmd: data, + } +} + +func getAction(cmd proto.Message) *flight.Action { + var anycmd anypb.Any + anycmd.MarshalFrom(cmd) + + data, _ := proto.Marshal(&anycmd) + return &flight.Action{Body: data} +} + +func (s *FlightSqlClientSuite) SetupTest() { + s.mockClient = FlightServiceClientMock{} + s.sqlClient.Client = &s.mockClient +} + +func (s *FlightSqlClientSuite) TearDownTest() { + s.mockClient.AssertExpectations(s.T()) +} + +var emptyFlightInfo flight.FlightInfo + +func (s *FlightSqlClientSuite) TestGetCatalogs() { + var cmd pb.CommandGetCatalogs + desc := getDesc(&cmd) + + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + info, err := s.sqlClient.GetCatalogs(context.Background(), s.callOpts...) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +func (s *FlightSqlClientSuite) TestGetDBSchemas() { + var ( + schemaFilterPattern = "schema_filter_pattern" + catalog = "catalog" + ) + + cmd := &pb.CommandGetDbSchemas{ + Catalog: &catalog, + DbSchemaFilterPattern: &schemaFilterPattern, + } + desc := getDesc(cmd) + + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + info, err := s.sqlClient.GetDBSchemas(context.Background(), (*flightsql.GetDBSchemasOpts)(cmd), s.callOpts...) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +func (s *FlightSqlClientSuite) TestGetTables() { + var ( + catalog = "catalog" + schemaFilterPattern = "schema_filter_pattern" + tableNameFilterPattern = "table_name_filter_pattern" + includeSchema = true + tableTypes = []string{"type1", "type2"} + ) + + cmd := &pb.CommandGetTables{ + Catalog: &catalog, + DbSchemaFilterPattern: &schemaFilterPattern, + TableNameFilterPattern: &tableNameFilterPattern, + IncludeSchema: includeSchema, + TableTypes: tableTypes, + } + desc := getDesc(cmd) + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + info, err := s.sqlClient.GetTables(context.Background(), (*flightsql.GetTablesOpts)(cmd), s.callOpts...) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +func (s *FlightSqlClientSuite) TestGetTableTypes() { + var cmd pb.CommandGetTableTypes + desc := getDesc(&cmd) + + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + info, err := s.sqlClient.GetTableTypes(context.Background(), s.callOpts...) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +func (s *FlightSqlClientSuite) TestGetTypeInfo() { + var cmd pb.CommandGetXdbcTypeInfo + desc := getDesc(&cmd) + + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + info, err := s.sqlClient.GetXdbcTypeInfo(context.Background(), nil, s.callOpts...) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +func (s *FlightSqlClientSuite) TestGetExported() { + var ( + catalog = "catalog" + schema = "schema" + table = "table" + ) + + cmd := &pb.CommandGetExportedKeys{ + Catalog: &catalog, + DbSchema: &schema, + Table: table, + } + desc := getDesc(cmd) + + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + info, err := s.sqlClient.GetExportedKeys(context.Background(), flightsql.TableRef{&catalog, &schema, table}, s.callOpts...) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +func (s *FlightSqlClientSuite) TestGetImported() { + var ( + schema = "schema" + table = "table" + ) + + cmd := &pb.CommandGetImportedKeys{ + DbSchema: &schema, + Table: table, + } + desc := getDesc(cmd) + + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + info, err := s.sqlClient.GetImportedKeys(context.Background(), flightsql.TableRef{nil, &schema, table}, s.callOpts...) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +func (s *FlightSqlClientSuite) TestGetPrimary() { + var ( + catalog = "catalog" + table = "table" + ) + + cmd := &pb.CommandGetPrimaryKeys{ + Catalog: &catalog, + Table: table, + } + desc := getDesc(cmd) + + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + info, err := s.sqlClient.GetPrimaryKeys(context.Background(), flightsql.TableRef{&catalog, nil, table}, s.callOpts...) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +func (s *FlightSqlClientSuite) TestGetCrossReference() { + var ( + pkCatalog = "pk_catalog" + pkSchema = "pk_schema" + pkTable = "pk_table" + fkCatalog = "fk_catalog" + fkSchema = "fk_schema" + fkTable = "fk_table" + ) + + cmd := &pb.CommandGetCrossReference{ + PkCatalog: &pkCatalog, + PkDbSchema: &pkSchema, + PkTable: pkTable, + FkCatalog: &fkCatalog, + FkDbSchema: &fkSchema, + FkTable: fkTable, + } + desc := getDesc(cmd) + + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + info, err := s.sqlClient.GetCrossReference(context.Background(), + flightsql.TableRef{&pkCatalog, &pkSchema, pkTable}, + flightsql.TableRef{&fkCatalog, &fkSchema, fkTable}, s.callOpts...) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +func (s *FlightSqlClientSuite) TestExecute() { + var query = "query" + + cmd := &pb.CommandStatementQuery{Query: query} + desc := getDesc(cmd) + + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + info, err := s.sqlClient.Execute(context.Background(), query, s.callOpts...) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +type mockDoActionClient struct { + mockGrpcClientStream +} + +func (m *mockDoActionClient) Recv() (*pb.Result, error) { + args := m.Called() + return args.Get(0).(*pb.Result), args.Error(1) +} + +type mockDoPutClient struct { + mockGrpcClientStream +} + +func (m *mockDoPutClient) Send(fd *flight.FlightData) error { + return m.Called(fd).Error(0) +} + +func (m *mockDoPutClient) Recv() (*pb.PutResult, error) { + args := m.Called() + return args.Get(0).(*pb.PutResult), args.Error(1) +} + +func (s *FlightSqlClientSuite) TestPreparedStatementExecute() { + const query = "query" + + cmd := &pb.ActionCreatePreparedStatementRequest{Query: query} + action := getAction(cmd) + action.Type = flightsql.CreatePreparedStatementActionType + closeAct := getAction(&pb.ActionClosePreparedStatementRequest{PreparedStatementHandle: []byte(query)}) + closeAct.Type = flightsql.ClosePreparedStatementActionType + + rsp := &mockDoActionClient{} + defer rsp.AssertExpectations(s.T()) + + result := &pb.ActionCreatePreparedStatementResult{PreparedStatementHandle: []byte(query)} + var out anypb.Any + out.MarshalFrom(result) + data, _ := proto.Marshal(&out) + rsp.On("Recv").Return(&pb.Result{Body: data}, nil) + + s.mockClient.On("DoAction", flightsql.CreatePreparedStatementActionType, action.Body, s.callOpts). + Return(rsp, nil) + s.mockClient.On("DoAction", flightsql.ClosePreparedStatementActionType, closeAct.Body, s.callOpts). + Return(rsp, nil) + + infoCmd := &pb.CommandPreparedStatementQuery{PreparedStatementHandle: []byte(query)} + desc := getDesc(infoCmd) + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + + prepared, err := s.sqlClient.Prepare(context.TODO(), memory.DefaultAllocator, query, s.callOpts...) + s.NoError(err) + defer prepared.Close(context.TODO()) + + info, err := prepared.Execute(context.TODO()) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +func (s *FlightSqlClientSuite) TestPreparedStatementExecuteParamBinding() { + const query = "query" + + // create and close actions + cmd := &pb.ActionCreatePreparedStatementRequest{Query: query} + action := getAction(cmd) + action.Type = flightsql.CreatePreparedStatementActionType + closeAct := getAction(&pb.ActionClosePreparedStatementRequest{PreparedStatementHandle: []byte(query)}) + closeAct.Type = flightsql.ClosePreparedStatementActionType + + // results from createprepared statement + result := &pb.ActionCreatePreparedStatementResult{ + PreparedStatementHandle: []byte(query), + } + schema := arrow.NewSchema([]arrow.Field{{Name: "id", Type: arrow.PrimitiveTypes.Int64, Nullable: true}}, nil) + result.ParameterSchema = flight.SerializeSchema(schema, memory.DefaultAllocator) + + // mocked client stream + var out anypb.Any + out.MarshalFrom(result) + data, _ := proto.Marshal(&out) + rsp := &mockDoActionClient{} + defer rsp.AssertExpectations(s.T()) + rsp.On("Recv").Return(&pb.Result{Body: data}, nil) + + // expect two actions: one to create and one to close the prepared statement + s.mockClient.On("DoAction", flightsql.CreatePreparedStatementActionType, action.Body, s.callOpts).Return(rsp, nil) + s.mockClient.On("DoAction", flightsql.ClosePreparedStatementActionType, closeAct.Body, s.callOpts).Return(rsp, nil) + + expectedDesc := getDesc(&pb.CommandPreparedStatementQuery{PreparedStatementHandle: []byte(query)}) + + // mocked client stream for DoPut + mockedPut := &mockDoPutClient{} + s.mockClient.On("DoPut", s.callOpts).Return(mockedPut, nil) + mockedPut.On("Send", mock.MatchedBy(func(fd *flight.FlightData) bool { + return proto.Equal(expectedDesc, fd.FlightDescriptor) + })).Return(nil).Twice() // first sends schema message, second sends data + mockedPut.On("CloseSend").Return(nil) + mockedPut.On("Recv").Return((*pb.PutResult)(nil), nil) + + infoCmd := &pb.CommandPreparedStatementQuery{PreparedStatementHandle: []byte(query)} + desc := getDesc(infoCmd) + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + + prepared, err := s.sqlClient.Prepare(context.TODO(), memory.DefaultAllocator, query, s.callOpts...) + s.NoError(err) + defer prepared.Close(context.TODO()) + + paramSchema := prepared.ParameterSchema() + rec, _, err := array.RecordFromJSON(memory.DefaultAllocator, paramSchema, strings.NewReader(`[{"id": 1}]`)) + s.NoError(err) + defer rec.Release() + + prepared.SetParameters(rec) + info, err := prepared.Execute(context.TODO()) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +func (s *FlightSqlClientSuite) TestExecuteUpdate() { + const query = "query" + + cmd := &pb.CommandStatementUpdate{Query: query} + desc := getDesc(cmd) + result := &pb.DoPutUpdateResult{RecordCount: 100} + resdata, _ := proto.Marshal(result) + + mockedPut := &mockDoPutClient{} + mockedPut.On("Send", mock.MatchedBy(func(fd *flight.FlightData) bool { + return proto.Equal(desc, fd.FlightDescriptor) + })).Return(nil) + mockedPut.On("CloseSend").Return(nil) + mockedPut.On("Recv").Return(&pb.PutResult{AppMetadata: resdata}, nil) + s.mockClient.On("DoPut", s.callOpts).Return(mockedPut, nil) + + num, err := s.sqlClient.ExecuteUpdate(context.TODO(), query, s.callOpts...) + s.NoError(err) + s.EqualValues(100, num) +} + +func (s *FlightSqlClientSuite) TestGetSqlInfo() { + sqlInfo := []flightsql.SqlInfo{ + flightsql.SqlInfoFlightSqlServerName, + flightsql.SqlInfoFlightSqlServerVersion, + flightsql.SqlInfoFlightSqlServerArrowVersion, + } + + cmd := &pb.CommandGetSqlInfo{Info: make([]uint32, len(sqlInfo))} + for i, info := range sqlInfo { + cmd.Info[i] = uint32(info) + } + desc := getDesc(cmd) + + s.mockClient.On("GetFlightInfo", desc.Type, desc.Cmd, s.callOpts).Return(&emptyFlightInfo, nil) + info, err := s.sqlClient.GetSqlInfo(context.TODO(), sqlInfo, s.callOpts...) + s.NoError(err) + s.Equal(&emptyFlightInfo, info) +} + +func TestFlightSqlClient(t *testing.T) { + suite.Run(t, new(FlightSqlClientSuite)) +} diff --git a/go/arrow/flight/flightsql/column_metadata.go b/go/arrow/flight/flightsql/column_metadata.go new file mode 100644 index 0000000000000..59c133757d5db --- /dev/null +++ b/go/arrow/flight/flightsql/column_metadata.go @@ -0,0 +1,212 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package flightsql + +import ( + "strconv" + + "github.com/apache/arrow/go/v10/arrow" +) + +const ( + boolTrueStr = "1" + boolFalseStr = "0" +) + +func boolToStr(v bool) string { + if v { + return boolTrueStr + } + return boolFalseStr +} + +func strToBool(v string) bool { + return v == boolTrueStr +} + +// Metadata Key Constants +const ( + CatalogNameKey = "ARROW:FLIGHT:SQL:CATALOG_NAME" + SchemaNameKey = "ARROW:FLIGHT:SQL:SCHEMA_NAME" + TableNameKey = "ARROW:FLIGHT:SQL:TABLE_NAME" + TypeNameKey = "ARROW:FLIGHT:SQL:TYPE_NAME" + PrecisionKey = "ARROW:FLIGHT:SQL:PRECISION" + ScaleKey = "ARROW:FLIGHT:SQL:SCALE" + IsAutoIncrementKey = "ARROW:FLIGHT:SQL:IS_AUTO_INCREMENT" + IsCaseSensitiveKey = "ARROW:FLIGHT:SQL:IS_CASE_SENSITIVE" + IsReadOnlyKey = "ARROW:FLIGHT:SQL:IS_READ_ONLY" + IsSearchableKey = "ARROW:FLIGHT:SQL:IS_SEARCHABLE" +) + +// ColumnMetadata is a helper object for managing and querying the +// standard SQL Column metadata using the expected Metadata Keys. +// It can be created by just Wrapping an existing *arrow.Metadata. +// +// Each of the methods return a value and a boolean indicating if it +// was set in the metadata or not. +type ColumnMetadata struct { + Data *arrow.Metadata +} + +func (c *ColumnMetadata) findStrVal(key string) (string, bool) { + idx := c.Data.FindKey(CatalogNameKey) + if idx == -1 { + return "", false + } + return c.Data.Values()[idx], true +} + +func (c *ColumnMetadata) findBoolVal(key string) (bool, bool) { + idx := c.Data.FindKey(CatalogNameKey) + if idx == -1 { + return false, false + } + return strToBool(c.Data.Values()[idx]), true +} + +func (c *ColumnMetadata) findInt32Val(key string) (int32, bool) { + idx := c.Data.FindKey(CatalogNameKey) + if idx == -1 { + return 0, false + } + v, err := strconv.ParseInt(c.Data.Values()[idx], 10, 32) + if err != nil { + return 0, false + } + return int32(v), true +} + +func (c *ColumnMetadata) CatalogName() (string, bool) { + return c.findStrVal(CatalogNameKey) +} + +func (c *ColumnMetadata) SchemaName() (string, bool) { + return c.findStrVal(SchemaNameKey) +} + +func (c *ColumnMetadata) TableName() (string, bool) { + return c.findStrVal(TableNameKey) +} + +func (c *ColumnMetadata) TypeName() (string, bool) { + return c.findStrVal(TypeNameKey) +} + +func (c *ColumnMetadata) Precision() (int32, bool) { + return c.findInt32Val(PrecisionKey) +} + +func (c *ColumnMetadata) Scale() (int32, bool) { + return c.findInt32Val(ScaleKey) +} + +func (c *ColumnMetadata) IsAutoIncrement() (bool, bool) { + return c.findBoolVal(IsAutoIncrementKey) +} + +func (c *ColumnMetadata) IsCaseSensitive() (bool, bool) { + return c.findBoolVal(IsCaseSensitiveKey) +} + +func (c *ColumnMetadata) IsReadOnly() (bool, bool) { + return c.findBoolVal(IsReadOnlyKey) +} + +func (c *ColumnMetadata) IsSearchable() (bool, bool) { + return c.findBoolVal(IsSearchableKey) +} + +// ColumnMetadataBuilder is a convenience builder for constructing +// sql column metadata using the expected standard metadata keys. +// All methods return the builder itself so it can be chained +// to easily construct a final metadata object. +type ColumnMetadataBuilder struct { + keys, vals []string +} + +func NewColumnMetadataBuilder() *ColumnMetadataBuilder { + return &ColumnMetadataBuilder{make([]string, 0), make([]string, 0)} +} + +func (c *ColumnMetadataBuilder) Build() ColumnMetadata { + md := c.Metadata() + return ColumnMetadata{&md} +} + +func (c *ColumnMetadataBuilder) Metadata() arrow.Metadata { + return arrow.NewMetadata(c.keys, c.vals) +} + +func (c *ColumnMetadataBuilder) CatalogName(name string) *ColumnMetadataBuilder { + c.keys = append(c.keys, CatalogNameKey) + c.vals = append(c.vals, name) + return c +} + +func (c *ColumnMetadataBuilder) SchemaName(name string) *ColumnMetadataBuilder { + c.keys = append(c.keys, SchemaNameKey) + c.vals = append(c.vals, name) + return c +} + +func (c *ColumnMetadataBuilder) TableName(name string) *ColumnMetadataBuilder { + c.keys = append(c.keys, TableNameKey) + c.vals = append(c.vals, name) + return c +} + +func (c *ColumnMetadataBuilder) TypeName(name string) *ColumnMetadataBuilder { + c.keys = append(c.keys, TypeNameKey) + c.vals = append(c.vals, name) + return c +} + +func (c *ColumnMetadataBuilder) Precision(prec int32) *ColumnMetadataBuilder { + c.keys = append(c.keys, PrecisionKey) + c.vals = append(c.vals, strconv.Itoa(int(prec))) + return c +} + +func (c *ColumnMetadataBuilder) Scale(prec int32) *ColumnMetadataBuilder { + c.keys = append(c.keys, ScaleKey) + c.vals = append(c.vals, strconv.Itoa(int(prec))) + return c +} + +func (c *ColumnMetadataBuilder) IsAutoIncrement(v bool) *ColumnMetadataBuilder { + c.keys = append(c.keys, IsAutoIncrementKey) + c.vals = append(c.vals, boolToStr(v)) + return c +} + +func (c *ColumnMetadataBuilder) IsCaseSensitive(v bool) *ColumnMetadataBuilder { + c.keys = append(c.keys, IsCaseSensitiveKey) + c.vals = append(c.vals, boolToStr(v)) + return c +} + +func (c *ColumnMetadataBuilder) IsReadOnly(v bool) *ColumnMetadataBuilder { + c.keys = append(c.keys, IsReadOnlyKey) + c.vals = append(c.vals, boolToStr(v)) + return c +} + +func (c *ColumnMetadataBuilder) IsSearchable(v bool) *ColumnMetadataBuilder { + c.keys = append(c.keys, IsSearchableKey) + c.vals = append(c.vals, boolToStr(v)) + return c +} diff --git a/go/arrow/flight/flightsql/schema_ref/reference_schemas.go b/go/arrow/flight/flightsql/schema_ref/reference_schemas.go new file mode 100644 index 0000000000000..7a4a14064d540 --- /dev/null +++ b/go/arrow/flight/flightsql/schema_ref/reference_schemas.go @@ -0,0 +1,106 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package schema_ref contains the expected reference Schemas to be used +// by FlightSQL servers and clients. +package schema_ref + +import "github.com/apache/arrow/go/v10/arrow" + +var ( + Catalogs = arrow.NewSchema( + []arrow.Field{{Name: "catalog_name", Type: arrow.BinaryTypes.String}}, nil) + DBSchemas = arrow.NewSchema([]arrow.Field{ + {Name: "catalog_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "db_schema_name", Type: arrow.BinaryTypes.String}, + }, nil) + Tables = arrow.NewSchema([]arrow.Field{ + {Name: "catalog_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "db_schema_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "table_name", Type: arrow.BinaryTypes.String}, + {Name: "table_type", Type: arrow.BinaryTypes.String}, + }, nil) + TablesWithIncludedSchema = arrow.NewSchema([]arrow.Field{ + {Name: "catalog_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "db_schema_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "table_name", Type: arrow.BinaryTypes.String}, + {Name: "table_type", Type: arrow.BinaryTypes.String}, + {Name: "table_schema", Type: arrow.BinaryTypes.Binary}, + }, nil) + TableTypes = arrow.NewSchema([]arrow.Field{ + {Name: "table_type", Type: arrow.BinaryTypes.String}, + }, nil) + PrimaryKeys = arrow.NewSchema([]arrow.Field{ + {Name: "catalog_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "db_schema_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "table_name", Type: arrow.BinaryTypes.String}, + {Name: "column_name", Type: arrow.BinaryTypes.String}, + {Name: "key_sequence", Type: arrow.PrimitiveTypes.Int32}, + {Name: "key_name", Type: arrow.BinaryTypes.String, Nullable: true}, + }, nil) + ImportedExportedKeysAndCrossReference = arrow.NewSchema([]arrow.Field{ + {Name: "pk_catalog_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "pk_db_schema_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "pk_table_name", Type: arrow.BinaryTypes.String, Nullable: false}, + {Name: "pk_column_name", Type: arrow.BinaryTypes.String, Nullable: false}, + {Name: "fk_catalog_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "fk_db_schema_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "fk_table_name", Type: arrow.BinaryTypes.String, Nullable: false}, + {Name: "fk_column_name", Type: arrow.BinaryTypes.String, Nullable: false}, + {Name: "key_sequence", Type: arrow.PrimitiveTypes.Int32, Nullable: false}, + {Name: "fk_key_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "pk_key_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "update_rule", Type: arrow.PrimitiveTypes.Uint8, Nullable: false}, + {Name: "delete_rule", Type: arrow.PrimitiveTypes.Uint8, Nullable: false}, + }, nil) + ImportedKeys = ImportedExportedKeysAndCrossReference + ExportedKeys = ImportedExportedKeysAndCrossReference + CrossReference = ImportedExportedKeysAndCrossReference + SqlInfo = arrow.NewSchema([]arrow.Field{ + {Name: "info_name", Type: arrow.PrimitiveTypes.Uint32}, + {Name: "value", Type: arrow.DenseUnionOf([]arrow.Field{ + {Name: "string_value", Type: arrow.BinaryTypes.String}, + {Name: "bool_value", Type: arrow.FixedWidthTypes.Boolean}, + {Name: "bigint_value", Type: arrow.PrimitiveTypes.Int64}, + {Name: "int32_bitmask", Type: arrow.PrimitiveTypes.Int32}, + {Name: "string_list", Type: arrow.ListOf(arrow.BinaryTypes.String)}, + {Name: "int32_to_int32_list_map", + Type: arrow.MapOf(arrow.PrimitiveTypes.Int32, + arrow.ListOf(arrow.PrimitiveTypes.Int32))}, + }, []arrow.UnionTypeCode{0, 1, 2, 3, 4, 5})}, + }, nil) + XdbcTypeInfo = arrow.NewSchema([]arrow.Field{ + {Name: "type_name", Type: arrow.BinaryTypes.String, Nullable: false}, + {Name: "data_type", Type: arrow.PrimitiveTypes.Int32, Nullable: false}, + {Name: "column_size", Type: arrow.PrimitiveTypes.Int32, Nullable: true}, + {Name: "literal_prefix", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "literal_suffix", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "create_params", Type: arrow.ListOfField(arrow.Field{Name: "item", Type: arrow.BinaryTypes.String, Nullable: false}), Nullable: true}, + {Name: "nullable", Type: arrow.PrimitiveTypes.Int32, Nullable: false}, + {Name: "case_sensitive", Type: arrow.FixedWidthTypes.Boolean, Nullable: false}, + {Name: "searchable", Type: arrow.PrimitiveTypes.Int32, Nullable: false}, + {Name: "unsigned_attribute", Type: arrow.FixedWidthTypes.Boolean, Nullable: true}, + {Name: "fixed_prec_scale", Type: arrow.FixedWidthTypes.Boolean, Nullable: false}, + {Name: "auto_increment", Type: arrow.FixedWidthTypes.Boolean, Nullable: true}, + {Name: "local_type_name", Type: arrow.BinaryTypes.String, Nullable: true}, + {Name: "minimum_scale", Type: arrow.PrimitiveTypes.Int32, Nullable: true}, + {Name: "maximum_scale", Type: arrow.PrimitiveTypes.Int32, Nullable: true}, + {Name: "sql_data_type", Type: arrow.PrimitiveTypes.Int32, Nullable: false}, + {Name: "datetime_subcode", Type: arrow.PrimitiveTypes.Int32, Nullable: true}, + {Name: "num_prec_radix", Type: arrow.PrimitiveTypes.Int32, Nullable: true}, + {Name: "interval_precision", Type: arrow.PrimitiveTypes.Int32, Nullable: true}, + }, nil) +) diff --git a/go/arrow/flight/flightsql/server.go b/go/arrow/flight/flightsql/server.go new file mode 100644 index 0000000000000..3823b84c1d2ad --- /dev/null +++ b/go/arrow/flight/flightsql/server.go @@ -0,0 +1,718 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package flightsql + +import ( + "context" + "fmt" + + "github.com/apache/arrow/go/v10/arrow" + "github.com/apache/arrow/go/v10/arrow/array" + "github.com/apache/arrow/go/v10/arrow/flight" + "github.com/apache/arrow/go/v10/arrow/flight/flightsql/schema_ref" + pb "github.com/apache/arrow/go/v10/arrow/flight/internal/flight" + "github.com/apache/arrow/go/v10/arrow/internal/debug" + "github.com/apache/arrow/go/v10/arrow/ipc" + "github.com/apache/arrow/go/v10/arrow/memory" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/anypb" +) + +// the following interfaces wrap the Protobuf commands to avoid +// exposing the Protobuf types themselves in the API. + +// StatementQuery represents a Sql Query +type StatementQuery interface { + GetQuery() string +} + +// StatementUpdate represents a SQL update query +type StatementUpdate interface { + GetQuery() string +} + +// StatementQueryTicket represents a request to execute a query +type StatementQueryTicket interface { + // GetStatementHandle returns the server-generated opaque + // identifier for the query + GetStatementHandle() []byte +} + +// PreparedStatementQuery represents a prepared query statement +type PreparedStatementQuery interface { + // GetPreparedStatementHandle returns the server-generated opaque + // identifier for the statement + GetPreparedStatementHandle() []byte +} + +// PreparedStatementUpdate represents a prepared update statement +type PreparedStatementUpdate interface { + // GetPreparedStatementHandle returns the server-generated opaque + // identifier for the statement + GetPreparedStatementHandle() []byte +} + +// ActionClosePreparedStatementRequest represents a request to close +// a prepared statement +type ActionClosePreparedStatementRequest interface { + // GetPreparedStatementHandle returns the server-generated opaque + // identifier for the statement + GetPreparedStatementHandle() []byte +} + +// ActionCreatePreparedStatementRequest represents a request to construct +// a new prepared statement +type ActionCreatePreparedStatementRequest interface { + GetQuery() string +} + +// ActionCreatePreparedStatementResult is the result of creating a new +// prepared statement, optionally including the dataset and parameter +// schemas. +type ActionCreatePreparedStatementResult struct { + Handle []byte + DatasetSchema *arrow.Schema + ParameterSchema *arrow.Schema +} + +type getXdbcTypeInfo struct { + *pb.CommandGetXdbcTypeInfo +} + +func (c *getXdbcTypeInfo) GetDataType() *int32 { return c.DataType } + +// GetXdbcTypeInfo represents a request for SQL Data Type information +type GetXdbcTypeInfo interface { + // GetDataType returns either nil (get for all types) + // or a specific SQL type ID to fetch information about. + GetDataType() *int32 +} + +// GetSqlInfo represents a request for SQL Information +type GetSqlInfo interface { + // GetInfo returns a slice of SqlInfo ids to return information about + GetInfo() []uint32 +} + +type getDBSchemas struct { + *pb.CommandGetDbSchemas +} + +func (c *getDBSchemas) GetCatalog() *string { return c.Catalog } +func (c *getDBSchemas) GetDBSchemaFilterPattern() *string { return c.DbSchemaFilterPattern } + +// GetDBSchemas represents a request for list of database schemas +type GetDBSchemas interface { + GetCatalog() *string + GetDBSchemaFilterPattern() *string +} + +type getTables struct { + *pb.CommandGetTables +} + +func (c *getTables) GetCatalog() *string { return c.Catalog } +func (c *getTables) GetDBSchemaFilterPattern() *string { return c.DbSchemaFilterPattern } +func (c *getTables) GetTableNameFilterPattern() *string { return c.TableNameFilterPattern } + +// GetTables represents a request to list the database's tables +type GetTables interface { + GetCatalog() *string + GetDBSchemaFilterPattern() *string + GetTableNameFilterPattern() *string + GetTableTypes() []string + GetIncludeSchema() bool +} + +// BaseServer must be embedded into any FlightSQL Server implementation +// and provides default implementations of all methods returning an +// unimplemented error if called. This allows consumers to gradually +// implement methods as they want instead of requiring all consumers to +// boilerplate the same "unimplemented" methods. +// +// The base implementation also contains handling for registering sql info +// and serving it up in response to GetSqlInfo requests. +type BaseServer struct { + sqlInfoToResult SqlInfoResultMap + // Alloc allows specifying a particular allocator to use for any + // allocations done by the base implementation. + // Will use memory.DefaultAlloctor if nil + Alloc memory.Allocator +} + +func (BaseServer) mustEmbedBaseServer() {} + +// RegisterSqlInfo registers a specific result to return for a given sqlinfo +// id. The result must be one of the following types: string, bool, int64, +// int32, []string, or map[int32][]int32. +// +// Once registered, this value will be returned for any SqlInfo requests. +func (b *BaseServer) RegisterSqlInfo(id SqlInfo, result interface{}) error { + switch result.(type) { + case string, bool, int64, int32, []string, map[int32][]int32: + b.sqlInfoToResult[uint32(id)] = result + default: + return fmt.Errorf("invalid sql info type '%T' registered for id: %d", result, id) + } + return nil +} + +func (BaseServer) GetFlightInfoStatement(context.Context, StatementQuery, *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoStatement not implemented") +} + +func (BaseServer) DoGetStatement(context.Context, StatementQueryTicket) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return nil, nil, status.Errorf(codes.Unimplemented, "DoGetStatement not implemented") +} + +func (BaseServer) GetFlightInfoPreparedStatement(context.Context, PreparedStatementQuery, *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoPreparedStatement not implemented") +} + +func (BaseServer) DoGetPreparedStatement(context.Context, PreparedStatementQuery) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return nil, nil, status.Errorf(codes.Unimplemented, "DoGetPreparedStatement not implemented") +} + +func (BaseServer) GetFlightInfoCatalogs(context.Context, *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoCatalogs not implemented") +} + +func (BaseServer) DoGetCatalogs(context.Context) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return nil, nil, status.Errorf(codes.Unimplemented, "DoGetCatalogs not implemented") +} + +func (BaseServer) GetFlightInfoXdbcTypeInfo(context.Context, GetXdbcTypeInfo, *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoXdbcTypeInfo not implemented") +} + +func (BaseServer) DoGetXdbcTypeInfo(context.Context, GetXdbcTypeInfo) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return nil, nil, status.Errorf(codes.Unimplemented, "DoGetXdbcTypeInfo not implemented") +} + +// GetFlightInfoSqlInfo is a base implementation of GetSqlInfo by using any +// registered sqlinfo (by calling RegisterSqlInfo). Will return an error +// if there is no sql info registered, otherwise a FlightInfo for retrieving +// the Sql info. +func (b *BaseServer) GetFlightInfoSqlInfo(_ context.Context, _ GetSqlInfo, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + if len(b.sqlInfoToResult) == 0 { + return nil, status.Error(codes.NotFound, "no sql information available") + } + + if b.Alloc == nil { + b.Alloc = memory.DefaultAllocator + } + + return &flight.FlightInfo{ + Endpoint: []*flight.FlightEndpoint{{Ticket: &flight.Ticket{Ticket: desc.Cmd}}}, + FlightDescriptor: desc, + TotalRecords: -1, + TotalBytes: -1, + Schema: flight.SerializeSchema(schema_ref.SqlInfo, b.Alloc), + }, nil +} + +// DoGetSqlInfo returns a flight stream containing the list of sqlinfo results +func (b *BaseServer) DoGetSqlInfo(_ context.Context, cmd GetSqlInfo) (*arrow.Schema, <-chan flight.StreamChunk, error) { + if b.Alloc == nil { + b.Alloc = memory.DefaultAllocator + } + + bldr := array.NewRecordBuilder(b.Alloc, schema_ref.SqlInfo) + defer bldr.Release() + + nameFieldBldr := bldr.Field(0).(*array.Uint32Builder) + valFieldBldr := bldr.Field(1).(*array.DenseUnionBuilder) + + // doesn't take ownership, no calls to retain. so we don't need + // extra releases. + sqlInfoResultBldr := newSqlInfoResultBuilder(valFieldBldr) + + // populate both the nameFieldBldr and the values for each + // element on command.info. + // valueFieldBldr is populated depending on the data type + // since it's a dense union. The population for each + // data type is handled by the sqlInfoResultBuilder. + for _, info := range cmd.GetInfo() { + val, ok := b.sqlInfoToResult[info] + if !ok { + return nil, nil, status.Errorf(codes.NotFound, "no information for sql info number %d", info) + } + nameFieldBldr.Append(info) + sqlInfoResultBldr.Append(val) + } + + batch := bldr.NewRecord() + defer batch.Release() + debug.Assert(int(batch.NumRows()) == len(cmd.GetInfo()), "too many rows added to SqlInfo result") + + ch := make(chan flight.StreamChunk) + rdr, err := array.NewRecordReader(schema_ref.SqlInfo, []arrow.Record{batch}) + if err != nil { + return nil, nil, status.Errorf(codes.Internal, "error producing record response: %s", err.Error()) + } + + // StreamChunksFromReader will call release on the reader when done + go flight.StreamChunksFromReader(rdr, ch) + return schema_ref.SqlInfo, ch, nil +} + +func (BaseServer) GetFlightInfoSchemas(context.Context, GetDBSchemas, *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoSchemas not implemented") +} + +func (BaseServer) DoGetDBSchemas(context.Context, GetDBSchemas) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return nil, nil, status.Errorf(codes.Unimplemented, "DoGetDBSchemas not implemented") +} + +func (BaseServer) GetFlightInfoTables(context.Context, GetTables, *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoTables not implemented") +} + +func (BaseServer) DoGetTables(context.Context, GetTables) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return nil, nil, status.Errorf(codes.Unimplemented, "DoGetTables not implemented") +} + +func (BaseServer) GetFlightInfoTableTypes(context.Context, *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoTableTypes not implemented") +} + +func (BaseServer) DoGetTableTypes(context.Context) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return nil, nil, status.Errorf(codes.Unimplemented, "DoGetTableTypes not implemented") +} + +func (BaseServer) GetFlightInfoPrimaryKeys(context.Context, TableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return nil, status.Error(codes.Unimplemented, "GetFlightInfoPrimaryKeys not implemented") +} + +func (BaseServer) DoGetPrimaryKeys(context.Context, TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return nil, nil, status.Errorf(codes.Unimplemented, "DoGetPrimaryKeys not implemented") +} + +func (BaseServer) GetFlightInfoExportedKeys(context.Context, TableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return nil, status.Error(codes.Unimplemented, "GetFlightInfoExportedKeys not implemented") +} + +func (BaseServer) DoGetExportedKeys(context.Context, TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return nil, nil, status.Errorf(codes.Unimplemented, "DoGetExportedKeys not implemented") +} + +func (BaseServer) GetFlightInfoImportedKeys(context.Context, TableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return nil, status.Error(codes.Unimplemented, "GetFlightInfoImportedKeys not implemented") +} + +func (BaseServer) DoGetImportedKeys(context.Context, TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return nil, nil, status.Errorf(codes.Unimplemented, "DoGetImportedKeys not implemented") +} + +func (BaseServer) GetFlightInfoCrossReference(context.Context, CrossTableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return nil, status.Error(codes.Unimplemented, "GetFlightInfoCrossReference not implemented") +} + +func (BaseServer) DoGetCrossReference(context.Context, CrossTableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return nil, nil, status.Errorf(codes.Unimplemented, "DoGetCrossReference not implemented") +} + +func (BaseServer) CreatePreparedStatement(context.Context, ActionCreatePreparedStatementRequest) (res ActionCreatePreparedStatementResult, err error) { + return res, status.Error(codes.Unimplemented, "CreatePreparedStatement not implemented") +} + +func (BaseServer) ClosePreparedStatement(context.Context, ActionClosePreparedStatementRequest) error { + return status.Error(codes.Unimplemented, "ClosePreparedStatement not implemented") +} + +func (BaseServer) DoPutCommandStatementUpdate(context.Context, StatementUpdate) (int64, error) { + return 0, status.Error(codes.Unimplemented, "DoPutCommandStatementUpdate not implemented") +} +func (BaseServer) DoPutPreparedStatementQuery(context.Context, PreparedStatementQuery, flight.MessageReader, flight.MetadataWriter) error { + return status.Error(codes.Unimplemented, "DoPutPreparedStatementQuery not implemented") +} + +func (BaseServer) DoPutPreparedStatementUpdate(context.Context, PreparedStatementUpdate, flight.MessageReader) (int64, error) { + return 0, status.Error(codes.Unimplemented, "DoPutPreparedStatementUpdate not implemented") +} + +// Server is the required interface for a FlightSQL server. It is implemented by +// BaseServer which must be embedded in any implementation. The default +// implementation by BaseServer for each of these (except GetSqlInfo) +// +// GetFlightInfo* methods should return the FlightInfo object representing where +// to retrieve the results for a given request. +// +// DoGet* methods should return the Schema of the resulting stream along with +// a channel to retrieve stream chunks (each chunk is a record batch and optionally +// a descriptor and app metadata). The channel will be read from until it +// closes, sending each chunk on the stream. Since the channel is returned +// from the method, it should be populated within a goroutine to ensure +// there are no deadlocks. +type Server interface { + // GetFlightInfoStatement returns a FlightInfo for executing the requested sql query + GetFlightInfoStatement(context.Context, StatementQuery, *flight.FlightDescriptor) (*flight.FlightInfo, error) + // DoGetStatement returns a stream containing the query results for the + // requested statement handle that was populated by GetFlightInfoStatement + DoGetStatement(context.Context, StatementQueryTicket) (*arrow.Schema, <-chan flight.StreamChunk, error) + // GetFlightInfoPreparedStatement returns a FlightInfo for executing an already + // prepared statement with the provided statement handle. + GetFlightInfoPreparedStatement(context.Context, PreparedStatementQuery, *flight.FlightDescriptor) (*flight.FlightInfo, error) + // DoGetPreparedStatement returns a stream containing the results from executing + // a prepared statement query with the provided statement handle. + DoGetPreparedStatement(context.Context, PreparedStatementQuery) (*arrow.Schema, <-chan flight.StreamChunk, error) + // GetFlightInfoCatalogs returns a FlightInfo for the listing of all catalogs + GetFlightInfoCatalogs(context.Context, *flight.FlightDescriptor) (*flight.FlightInfo, error) + // DoGetCatalogs returns the stream containing the list of catalogs + DoGetCatalogs(context.Context) (*arrow.Schema, <-chan flight.StreamChunk, error) + // GetFlightInfoXdbcTypeInfo returns a FlightInfo for retrieving data type info + GetFlightInfoXdbcTypeInfo(context.Context, GetXdbcTypeInfo, *flight.FlightDescriptor) (*flight.FlightInfo, error) + // DoGetXdbcTypeInfo returns a stream containing the information about the + // requested supported datatypes + DoGetXdbcTypeInfo(context.Context, GetXdbcTypeInfo) (*arrow.Schema, <-chan flight.StreamChunk, error) + // GetFlightInfoSqlInfo returns a FlightInfo for retrieving SqlInfo from the server + GetFlightInfoSqlInfo(context.Context, GetSqlInfo, *flight.FlightDescriptor) (*flight.FlightInfo, error) + // DoGetSqlInfo returns a stream containing the list of SqlInfo results + DoGetSqlInfo(context.Context, GetSqlInfo) (*arrow.Schema, <-chan flight.StreamChunk, error) + // GetFlightInfoSchemas returns a FlightInfo for requesting a list of schemas + GetFlightInfoSchemas(context.Context, GetDBSchemas, *flight.FlightDescriptor) (*flight.FlightInfo, error) + // DoGetDBSchemas returns a stream containing the list of schemas + DoGetDBSchemas(context.Context, GetDBSchemas) (*arrow.Schema, <-chan flight.StreamChunk, error) + // GetFlightInfoTables returns a FlightInfo for listing the tables available + GetFlightInfoTables(context.Context, GetTables, *flight.FlightDescriptor) (*flight.FlightInfo, error) + // DoGetTables returns a stream containing the list of tables + DoGetTables(context.Context, GetTables) (*arrow.Schema, <-chan flight.StreamChunk, error) + // GetFlightInfoTableTypes returns a FlightInfo for retrieving a list + // of table types supported + GetFlightInfoTableTypes(context.Context, *flight.FlightDescriptor) (*flight.FlightInfo, error) + // DoGetTableTypes returns a stream containing the data related to the table types + DoGetTableTypes(context.Context) (*arrow.Schema, <-chan flight.StreamChunk, error) + // GetFlightInfoPrimaryKeys returns a FlightInfo for extracting information about primary keys + GetFlightInfoPrimaryKeys(context.Context, TableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error) + // DoGetPrimaryKeys returns a stream containing the data related to primary keys + DoGetPrimaryKeys(context.Context, TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) + // GetFlightInfoExportedKeys returns a FlightInfo for extracting information about foreign keys + GetFlightInfoExportedKeys(context.Context, TableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error) + // DoGetExportedKeys returns a stream containing the data related to foreign keys + DoGetExportedKeys(context.Context, TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) + // GetFlightInfoImportedKeys returns a FlightInfo for extracting information about imported keys + GetFlightInfoImportedKeys(context.Context, TableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error) + // DoGetImportedKeys returns a stream containing the data related to imported keys + DoGetImportedKeys(context.Context, TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) + // GetFlightInfoCrossReference returns a FlightInfo for extracting data related + // to primary and foreign keys + GetFlightInfoCrossReference(context.Context, CrossTableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error) + // DoGetCrossReference returns a stream of data related to foreign and primary keys + DoGetCrossReference(context.Context, CrossTableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) + // DoPutCommandStatementUpdate executes a sql update statement and returns + // the number of affected rows + DoPutCommandStatementUpdate(context.Context, StatementUpdate) (int64, error) + // CreatePreparedStatement constructs a prepared statement from a sql query + // and returns an opaque statement handle for use. + CreatePreparedStatement(context.Context, ActionCreatePreparedStatementRequest) (ActionCreatePreparedStatementResult, error) + // ClosePreparedStatement closes the prepared statement identified by the requested + // opaque statement handle. + ClosePreparedStatement(context.Context, ActionClosePreparedStatementRequest) error + // DoPutPreparedStatementQuery binds parameters to a given prepared statement + // identified by the provided statement handle. + // + // The provided MessageReader is a stream of record batches with optional + // app metadata and flight descriptors to represent the values to bind + // to the parameters. + // + // Currently anything written to the writer will be ignored. It is in the + // interface for potential future enhancements to avoid having to change + // the interface in the future. + DoPutPreparedStatementQuery(context.Context, PreparedStatementQuery, flight.MessageReader, flight.MetadataWriter) error + // DoPutPreparedStatementUpdate executes an update SQL Prepared statement + // for the specified statement handle. The reader allows providing a sequence + // of uploaded record batches to bind the parameters to. Returns the number + // of affected records. + DoPutPreparedStatementUpdate(context.Context, PreparedStatementUpdate, flight.MessageReader) (int64, error) + + mustEmbedBaseServer() +} + +// NewFlightServer constructs a FlightRPC server from the provided +// FlightSQL Server so that it can be passed to RegisterFlightService. +func NewFlightServer(srv Server) flight.FlightServer { + return &flightSqlServer{srv: srv, mem: memory.DefaultAllocator} +} + +// NewFlightServerWithAllocator constructs a FlightRPC server from +// the provided FlightSQL Server so that it can be passed to +// RegisterFlightService, setting the provided allocator into the server +// for use with any allocations necessary by the routing. +// +// Will default to memory.DefaultAllocator if mem is nil +func NewFlightServerWithAllocator(srv Server, mem memory.Allocator) flight.FlightServer { + if mem == nil { + mem = memory.DefaultAllocator + } + return &flightSqlServer{srv: srv, mem: mem} +} + +// flightSqlServer is a wrapper around a FlightSQL server interface to +// perform routing from FlightRPC to FlightSQL. +type flightSqlServer struct { + flight.BaseFlightServer + mem memory.Allocator + srv Server +} + +func (f *flightSqlServer) GetFlightInfo(ctx context.Context, request *flight.FlightDescriptor) (*flight.FlightInfo, error) { + var ( + anycmd anypb.Any + cmd proto.Message + err error + ) + if err = proto.Unmarshal(request.Cmd, &anycmd); err != nil { + return nil, status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error()) + } + + if cmd, err = anycmd.UnmarshalNew(); err != nil { + return nil, status.Errorf(codes.InvalidArgument, "could not unmarshal Any to a command type: %s", err.Error()) + } + + switch cmd := cmd.(type) { + case *pb.CommandStatementQuery: + return f.srv.GetFlightInfoStatement(ctx, cmd, request) + case *pb.CommandPreparedStatementQuery: + return f.srv.GetFlightInfoPreparedStatement(ctx, cmd, request) + case *pb.CommandGetCatalogs: + return f.srv.GetFlightInfoCatalogs(ctx, request) + case *pb.CommandGetDbSchemas: + return f.srv.GetFlightInfoSchemas(ctx, &getDBSchemas{cmd}, request) + case *pb.CommandGetTables: + return f.srv.GetFlightInfoTables(ctx, &getTables{cmd}, request) + case *pb.CommandGetTableTypes: + return f.srv.GetFlightInfoTableTypes(ctx, request) + case *pb.CommandGetXdbcTypeInfo: + return f.srv.GetFlightInfoXdbcTypeInfo(ctx, &getXdbcTypeInfo{cmd}, request) + case *pb.CommandGetSqlInfo: + return f.srv.GetFlightInfoSqlInfo(ctx, cmd, request) + case *pb.CommandGetPrimaryKeys: + return f.srv.GetFlightInfoPrimaryKeys(ctx, pkToTableRef(cmd), request) + case *pb.CommandGetExportedKeys: + return f.srv.GetFlightInfoExportedKeys(ctx, exkToTableRef(cmd), request) + case *pb.CommandGetImportedKeys: + return f.srv.GetFlightInfoImportedKeys(ctx, impkToTableRef(cmd), request) + case *pb.CommandGetCrossReference: + return f.srv.GetFlightInfoCrossReference(ctx, toCrossTableRef(cmd), request) + } + + return nil, status.Error(codes.InvalidArgument, "requested command is invalid") +} + +func (f *flightSqlServer) DoGet(request *flight.Ticket, stream flight.FlightService_DoGetServer) (err error) { + var ( + anycmd anypb.Any + cmd proto.Message + cc <-chan flight.StreamChunk + sc *arrow.Schema + ) + if err = proto.Unmarshal(request.Ticket, &anycmd); err != nil { + return status.Errorf(codes.InvalidArgument, "unable to parse ticket: %s", err.Error()) + } + + if cmd, err = anycmd.UnmarshalNew(); err != nil { + return status.Errorf(codes.InvalidArgument, "unable to unmarshal proto.Any: %s", err.Error()) + } + + switch cmd := cmd.(type) { + case *pb.TicketStatementQuery: + sc, cc, err = f.srv.DoGetStatement(stream.Context(), cmd) + case *pb.CommandPreparedStatementQuery: + sc, cc, err = f.srv.DoGetPreparedStatement(stream.Context(), cmd) + case *pb.CommandGetCatalogs: + sc, cc, err = f.srv.DoGetCatalogs(stream.Context()) + case *pb.CommandGetDbSchemas: + sc, cc, err = f.srv.DoGetDBSchemas(stream.Context(), &getDBSchemas{cmd}) + case *pb.CommandGetTables: + sc, cc, err = f.srv.DoGetTables(stream.Context(), &getTables{cmd}) + case *pb.CommandGetTableTypes: + sc, cc, err = f.srv.DoGetTableTypes(stream.Context()) + case *pb.CommandGetXdbcTypeInfo: + sc, cc, err = f.srv.DoGetXdbcTypeInfo(stream.Context(), &getXdbcTypeInfo{cmd}) + case *pb.CommandGetSqlInfo: + sc, cc, err = f.srv.DoGetSqlInfo(stream.Context(), cmd) + case *pb.CommandGetPrimaryKeys: + sc, cc, err = f.srv.DoGetPrimaryKeys(stream.Context(), pkToTableRef(cmd)) + case *pb.CommandGetExportedKeys: + sc, cc, err = f.srv.DoGetExportedKeys(stream.Context(), exkToTableRef(cmd)) + case *pb.CommandGetImportedKeys: + sc, cc, err = f.srv.DoGetImportedKeys(stream.Context(), impkToTableRef(cmd)) + case *pb.CommandGetCrossReference: + sc, cc, err = f.srv.DoGetCrossReference(stream.Context(), toCrossTableRef(cmd)) + default: + return status.Error(codes.InvalidArgument, "requested command is invalid") + } + + if err != nil { + return err + } + + wr := flight.NewRecordWriter(stream, ipc.WithSchema(sc)) + defer wr.Close() + + for chunk := range cc { + if chunk.Err != nil { + return err + } + + wr.SetFlightDescriptor(chunk.Desc) + if err = wr.WriteWithAppMetadata(chunk.Data, chunk.AppMetadata); err != nil { + return err + } + chunk.Data.Release() + } + + return err +} + +type putMetadataWriter struct { + stream flight.FlightService_DoPutServer +} + +func (p *putMetadataWriter) WriteMetadata(appMetadata []byte) error { + return p.stream.Send(&flight.PutResult{AppMetadata: appMetadata}) +} + +func (f *flightSqlServer) DoPut(stream flight.FlightService_DoPutServer) error { + rdr, err := flight.NewRecordReader(stream, ipc.WithAllocator(f.mem), ipc.WithDelayReadSchema(true)) + if err != nil { + return status.Errorf(codes.InvalidArgument, "failed to read input stream: %s", err.Error()) + } + defer rdr.Release() + + // flight descriptor should have come with the schema message + request := rdr.LatestFlightDescriptor() + + var ( + anycmd anypb.Any + cmd proto.Message + ) + if err = proto.Unmarshal(request.Cmd, &anycmd); err != nil { + return status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error()) + } + + if cmd, err = anycmd.UnmarshalNew(); err != nil { + return status.Errorf(codes.InvalidArgument, "could not unmarshal google.protobuf.Any: %s", err.Error()) + } + + switch cmd := cmd.(type) { + case *pb.CommandStatementUpdate: + recordCount, err := f.srv.DoPutCommandStatementUpdate(stream.Context(), cmd) + if err != nil { + return err + } + + result := pb.DoPutUpdateResult{RecordCount: recordCount} + out := &flight.PutResult{} + if out.AppMetadata, err = proto.Marshal(&result); err != nil { + return status.Errorf(codes.Internal, "failed to marshal PutResult: %s", err.Error()) + } + return stream.Send(out) + case *pb.CommandPreparedStatementQuery: + return f.srv.DoPutPreparedStatementQuery(stream.Context(), cmd, rdr, &putMetadataWriter{stream}) + case *pb.CommandPreparedStatementUpdate: + recordCount, err := f.srv.DoPutPreparedStatementUpdate(stream.Context(), cmd, rdr) + if err != nil { + return err + } + + result := pb.DoPutUpdateResult{RecordCount: recordCount} + out := &flight.PutResult{} + if out.AppMetadata, err = proto.Marshal(&result); err != nil { + return status.Errorf(codes.Internal, "failed to marshal PutResult: %s", err.Error()) + } + return stream.Send(out) + default: + return status.Error(codes.InvalidArgument, "the defined request is invalid") + } +} + +func (f *flightSqlServer) ListActions(_ *flight.Empty, stream flight.FlightService_ListActionsServer) error { + actions := []string{CreatePreparedStatementActionType, ClosePreparedStatementActionType} + + for _, a := range actions { + if err := stream.Send(&flight.ActionType{Type: a}); err != nil { + return err + } + } + return nil +} + +func (f *flightSqlServer) DoAction(cmd *flight.Action, stream flight.FlightService_DoActionServer) error { + var anycmd anypb.Any + + switch cmd.Type { + case CreatePreparedStatementActionType: + if err := proto.Unmarshal(cmd.Body, &anycmd); err != nil { + return status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error()) + } + + var ( + request pb.ActionCreatePreparedStatementRequest + result pb.ActionCreatePreparedStatementResult + ret pb.Result + ) + if err := anycmd.UnmarshalTo(&request); err != nil { + return status.Errorf(codes.InvalidArgument, "unable to unmarshal google.protobuf.Any: %s", err.Error()) + } + + output, err := f.srv.CreatePreparedStatement(stream.Context(), &request) + if err != nil { + return err + } + + result.PreparedStatementHandle = output.Handle + if output.DatasetSchema != nil { + result.DatasetSchema = flight.SerializeSchema(output.DatasetSchema, f.mem) + } + if output.ParameterSchema != nil { + result.ParameterSchema = flight.SerializeSchema(output.ParameterSchema, f.mem) + } + + if err := anycmd.MarshalFrom(&result); err != nil { + return status.Errorf(codes.Internal, "unable to marshal final response: %s", err.Error()) + } + + if ret.Body, err = proto.Marshal(&anycmd); err != nil { + return status.Errorf(codes.Internal, "unable to marshal result: %s", err.Error()) + } + return stream.Send(&ret) + case ClosePreparedStatementActionType: + if err := proto.Unmarshal(cmd.Body, &anycmd); err != nil { + return status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error()) + } + + var request pb.ActionClosePreparedStatementRequest + if err := anycmd.UnmarshalTo(&request); err != nil { + return status.Errorf(codes.InvalidArgument, "unable to unmarshal google.protobuf.Any: %s", err.Error()) + } + + if err := f.srv.ClosePreparedStatement(stream.Context(), &request); err != nil { + return err + } + + return stream.Send(&pb.Result{}) + default: + return status.Error(codes.InvalidArgument, "the defined request is invalid.") + } +} + +var ( + _ Server = (*BaseServer)(nil) +) diff --git a/go/arrow/flight/flightsql/server_test.go b/go/arrow/flight/flightsql/server_test.go new file mode 100644 index 0000000000000..ece7754bbbdb2 --- /dev/null +++ b/go/arrow/flight/flightsql/server_test.go @@ -0,0 +1,212 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package flightsql_test + +import ( + "context" + "strings" + "testing" + + "github.com/apache/arrow/go/v10/arrow/flight" + "github.com/apache/arrow/go/v10/arrow/flight/flightsql" + pb "github.com/apache/arrow/go/v10/arrow/flight/internal/flight" + "github.com/apache/arrow/go/v10/arrow/memory" + "github.com/stretchr/testify/suite" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/anypb" +) + +var dialOpts = []grpc.DialOption{grpc.WithTransportCredentials(insecure.NewCredentials())} + +type UnimplementedFlightSqlServerSuite struct { + suite.Suite + + s flight.Server + cl *flightsql.Client +} + +func (s *UnimplementedFlightSqlServerSuite) SetupSuite() { + s.s = flight.NewServerWithMiddleware(nil) + srv := flightsql.NewFlightServer(&flightsql.BaseServer{}) + s.s.RegisterFlightService(srv) + s.s.Init("localhost:0") + + go s.s.Serve() +} + +func (s *UnimplementedFlightSqlServerSuite) SetupTest() { + cl, err := flightsql.NewClient(s.s.Addr().String(), nil, nil, dialOpts...) + s.Require().NoError(err) + s.cl = cl +} + +func (s *UnimplementedFlightSqlServerSuite) TearDownTest() { + s.Require().NoError(s.cl.Close()) + s.cl = nil +} + +func (s *UnimplementedFlightSqlServerSuite) TearDownSuite() { + s.s.Shutdown() +} + +// the following test functions verify that the default base server will +// correctly route requests to the appropriate interface methods based on +// the descriptor types for DoPut/DoGet/DoAction + +func (s *UnimplementedFlightSqlServerSuite) TestExecute() { + info, err := s.cl.Execute(context.TODO(), "SELECT * FROM IRRELEVANT") + st, ok := status.FromError(err) + s.True(ok) + s.Equal(codes.Unimplemented, st.Code()) + s.Equal(st.Message(), "GetFlightInfoStatement not implemented") + s.Nil(info) +} + +func (s *UnimplementedFlightSqlServerSuite) TestGetTables() { + info, err := s.cl.GetTables(context.TODO(), &flightsql.GetTablesOpts{}) + st, ok := status.FromError(err) + s.True(ok) + s.Equal(codes.Unimplemented, st.Code()) + s.Equal(st.Message(), "GetFlightInfoTables not implemented") + s.Nil(info) +} + +func (s *UnimplementedFlightSqlServerSuite) TestGetTableTypes() { + info, err := s.cl.GetTableTypes(context.TODO()) + st, ok := status.FromError(err) + s.True(ok) + s.Equal(codes.Unimplemented, st.Code()) + s.Equal(st.Message(), "GetFlightInfoTableTypes not implemented") + s.Nil(info) +} + +func (s *UnimplementedFlightSqlServerSuite) TestGetPrimaryKeys() { + info, err := s.cl.GetPrimaryKeys(context.TODO(), flightsql.TableRef{}) + st, ok := status.FromError(err) + s.True(ok) + s.Equal(codes.Unimplemented, st.Code()) + s.Equal(st.Message(), "GetFlightInfoPrimaryKeys not implemented") + s.Nil(info) +} + +func (s *UnimplementedFlightSqlServerSuite) TestGetExportedKeys() { + info, err := s.cl.GetExportedKeys(context.TODO(), flightsql.TableRef{}) + st, ok := status.FromError(err) + s.True(ok) + s.Equal(codes.Unimplemented, st.Code()) + s.Equal(st.Message(), "GetFlightInfoExportedKeys not implemented") + s.Nil(info) +} + +func (s *UnimplementedFlightSqlServerSuite) TestGetImportedKeys() { + info, err := s.cl.GetImportedKeys(context.TODO(), flightsql.TableRef{}) + st, ok := status.FromError(err) + s.True(ok) + s.Equal(codes.Unimplemented, st.Code()) + s.Equal(st.Message(), "GetFlightInfoImportedKeys not implemented") + s.Nil(info) +} + +func (s *UnimplementedFlightSqlServerSuite) TestGetCrossReference() { + info, err := s.cl.GetCrossReference(context.TODO(), flightsql.TableRef{}, flightsql.TableRef{}) + st, ok := status.FromError(err) + s.True(ok) + s.Equal(codes.Unimplemented, st.Code()) + s.Equal(st.Message(), "GetFlightInfoCrossReference not implemented") + s.Nil(info) +} + +func (s *UnimplementedFlightSqlServerSuite) TestGetCatalogs() { + info, err := s.cl.GetCatalogs(context.TODO()) + st, ok := status.FromError(err) + s.True(ok) + s.Equal(codes.Unimplemented, st.Code()) + s.Equal(st.Message(), "GetFlightInfoCatalogs not implemented") + s.Nil(info) +} + +func (s *UnimplementedFlightSqlServerSuite) TestGetDBSchemas() { + info, err := s.cl.GetDBSchemas(context.TODO(), &flightsql.GetDBSchemasOpts{}) + st, ok := status.FromError(err) + s.True(ok) + s.Equal(codes.Unimplemented, st.Code()) + s.Equal(st.Message(), "GetFlightInfoSchemas not implemented") + s.Nil(info) +} + +func (s *UnimplementedFlightSqlServerSuite) TestGetTypeInfo() { + info, err := s.cl.GetXdbcTypeInfo(context.TODO(), nil) + st, ok := status.FromError(err) + s.True(ok) + s.Equal(codes.Unimplemented, st.Code()) + s.Equal(st.Message(), "GetFlightInfoXdbcTypeInfo not implemented") + s.Nil(info) +} + +func getTicket(cmd proto.Message) *flight.Ticket { + var anycmd anypb.Any + anycmd.MarshalFrom(cmd) + + data, _ := proto.Marshal(&anycmd) + return &flight.Ticket{ + Ticket: data, + } +} + +func (s *UnimplementedFlightSqlServerSuite) TestDoGet() { + tests := []struct { + name string + ticket proto.Message + }{ + {"DoGetStatement", &pb.TicketStatementQuery{}}, + {"DoGetPreparedStatement", &pb.CommandPreparedStatementQuery{}}, + {"DoGetCatalogs", &pb.CommandGetCatalogs{}}, + {"DoGetDBSchemas", &pb.CommandGetDbSchemas{}}, + {"DoGetTables", &pb.CommandGetTables{}}, + {"DoGetTableTypes", &pb.CommandGetTableTypes{}}, + {"DoGetXdbcTypeInfo", &pb.CommandGetXdbcTypeInfo{}}, + {"DoGetPrimaryKeys", &pb.CommandGetPrimaryKeys{}}, + {"DoGetExportedKeys", &pb.CommandGetExportedKeys{}}, + {"DoGetImportedKeys", &pb.CommandGetImportedKeys{}}, + {"DoGetCrossReference", &pb.CommandGetCrossReference{}}, + } + + for _, tt := range tests { + s.Run(tt.name, func() { + rdr, err := s.cl.DoGet(context.TODO(), getTicket(tt.ticket)) + s.Nil(rdr) + s.True(strings.HasSuffix(err.Error(), tt.name+" not implemented"), err.Error()) + }) + } +} + +func (s *UnimplementedFlightSqlServerSuite) TestDoAction() { + prep, err := s.cl.Prepare(context.TODO(), memory.DefaultAllocator, "IRRELEVANT") + s.Nil(prep) + st, ok := status.FromError(err) + s.True(ok) + s.Equal(codes.Unimplemented, st.Code()) + s.Equal(st.Message(), "CreatePreparedStatement not implemented") +} + +func TestBaseServer(t *testing.T) { + suite.Run(t, new(UnimplementedFlightSqlServerSuite)) +} diff --git a/go/arrow/flight/flightsql/sql_info.go b/go/arrow/flight/flightsql/sql_info.go new file mode 100644 index 0000000000000..687f10f6f5af6 --- /dev/null +++ b/go/arrow/flight/flightsql/sql_info.go @@ -0,0 +1,93 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package flightsql + +import ( + "github.com/apache/arrow/go/v10/arrow" + "github.com/apache/arrow/go/v10/arrow/array" +) + +const ( + strValIdx arrow.UnionTypeCode = iota + boolValIdx + bigintValIdx + int32BitMaskIdx + strListIdx + int32ToInt32ListIdx +) + +// sqlInfoResultBldr is a helper for building up the dense union response +// of a SqlInfo request. +type sqlInfoResultBldr struct { + valueBldr *array.DenseUnionBuilder + + strBldr *array.StringBuilder + boolBldr *array.BooleanBuilder + bigintBldr *array.Int64Builder + int32BitmaskBldr *array.Int32Builder + strListBldr *array.ListBuilder + int32Toint32ListBldr *array.MapBuilder +} + +func newSqlInfoResultBuilder(valueBldr *array.DenseUnionBuilder) *sqlInfoResultBldr { + return &sqlInfoResultBldr{ + valueBldr: valueBldr, + strBldr: valueBldr.Child(int(strValIdx)).(*array.StringBuilder), + boolBldr: valueBldr.Child(int(boolValIdx)).(*array.BooleanBuilder), + bigintBldr: valueBldr.Child(int(bigintValIdx)).(*array.Int64Builder), + int32BitmaskBldr: valueBldr.Child(int(int32BitMaskIdx)).(*array.Int32Builder), + strListBldr: valueBldr.Child(int(strListIdx)).(*array.ListBuilder), + int32Toint32ListBldr: valueBldr.Child(int(int32ToInt32ListIdx)).(*array.MapBuilder), + } +} + +func (s *sqlInfoResultBldr) Append(v interface{}) { + switch v := v.(type) { + case string: + s.valueBldr.Append(strValIdx) + s.strBldr.Append(v) + case bool: + s.valueBldr.Append(boolValIdx) + s.boolBldr.Append(v) + case int64: + s.valueBldr.Append(bigintValIdx) + s.bigintBldr.Append(v) + case int32: + s.valueBldr.Append(int32BitMaskIdx) + s.int32BitmaskBldr.Append(v) + case []string: + s.valueBldr.Append(strListIdx) + s.strListBldr.Append(true) + chld := s.strListBldr.ValueBuilder().(*array.StringBuilder) + chld.AppendValues(v, nil) + case map[int32][]int32: + s.valueBldr.Append(int32ToInt32ListIdx) + s.int32Toint32ListBldr.Append(true) + + kb := s.int32Toint32ListBldr.KeyBuilder().(*array.Int32Builder) + ib := s.int32Toint32ListBldr.ItemBuilder().(*array.ListBuilder) + ch := ib.ValueBuilder().(*array.Int32Builder) + + for key, val := range v { + kb.Append(key) + ib.Append(true) + for _, c := range val { + ch.Append(c) + } + } + } +} diff --git a/go/arrow/flight/flightsql/types.go b/go/arrow/flight/flightsql/types.go new file mode 100644 index 0000000000000..5e033d00ee322 --- /dev/null +++ b/go/arrow/flight/flightsql/types.go @@ -0,0 +1,745 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package flightsql + +import ( + pb "github.com/apache/arrow/go/v10/arrow/flight/internal/flight" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/anypb" +) + +// Constants for Action types +const ( + CreatePreparedStatementActionType = "CreatePreparedStatement" + ClosePreparedStatementActionType = "ClosePreparedStatement" +) + +func toCrossTableRef(cmd *pb.CommandGetCrossReference) CrossTableRef { + return CrossTableRef{ + PKRef: TableRef{ + Catalog: cmd.PkCatalog, + DBSchema: cmd.PkDbSchema, + Table: cmd.PkTable, + }, + FKRef: TableRef{ + Catalog: cmd.FkCatalog, + DBSchema: cmd.FkDbSchema, + Table: cmd.FkTable, + }, + } +} + +func pkToTableRef(cmd *pb.CommandGetPrimaryKeys) TableRef { + return TableRef{ + Catalog: cmd.Catalog, + DBSchema: cmd.DbSchema, + Table: cmd.Table, + } +} + +func exkToTableRef(cmd *pb.CommandGetExportedKeys) TableRef { + return TableRef{ + Catalog: cmd.Catalog, + DBSchema: cmd.DbSchema, + Table: cmd.Table, + } +} + +func impkToTableRef(cmd *pb.CommandGetImportedKeys) TableRef { + return TableRef{ + Catalog: cmd.Catalog, + DBSchema: cmd.DbSchema, + Table: cmd.Table, + } +} + +// CreateStatementQueryTicket is a helper that constructs a properly +// serialized TicketStatementQuery containing a given opaque binary handle +// for use with constructing a ticket to return from GetFlightInfoStatement. +func CreateStatementQueryTicket(handle []byte) ([]byte, error) { + query := &pb.TicketStatementQuery{StatementHandle: handle} + var ticket anypb.Any + ticket.MarshalFrom(query) + + return proto.Marshal(&ticket) +} + +type ( + // GetDBSchemasOpts contains the options to request Database Schemas: + // an optional Catalog and a Schema Name filter pattern. + GetDBSchemasOpts pb.CommandGetDbSchemas + // GetTablesOpts contains the options for retrieving a list of tables: + // optional Catalog, Schema filter pattern, Table name filter pattern, + // a filter of table types, and whether or not to include the schema + // in the response. + GetTablesOpts pb.CommandGetTables + + // SqlInfoResultMap is a mapping of SqlInfo ids to the desired response. + // This is part of a Server and used for registering responses to a + // SqlInfo request. + SqlInfoResultMap map[uint32]interface{} + + // TableRef is a helpful struct for referencing a specific Table + // by its catalog, schema, and table name. + TableRef struct { + // Catalog specifies the catalog this table belongs to. + // An empty string refers to tables without a catalog. + // If nil, can reference a table in any catalog. + Catalog *string + // DBSchema specifies the database schema the table belongs to. + // An empty string refers to a table which does not belong to + // a database schema. + // If nil, can reference a table in any database schema. + DBSchema *string + // Table is the name of the table that is being referenced. + Table string + } + + // CrossTableRef contains a reference to a Primary Key table + // and a Foreign Key table. + CrossTableRef struct { + PKRef TableRef + FKRef TableRef + } + + // since we are hiding the Protobuf internals in an internal + // package, we need to provide enum values for the SqlInfo enum here + SqlInfo uint32 +) + +// SqlInfo enum values +const ( + // Server Information + // Values [0-500): Provide information about the Flight SQL Server itself + + // Retrieves a UTF-8 string with the name of the Flight SQL Server. + SqlInfoFlightSqlServerName = SqlInfo(pb.SqlInfo_FLIGHT_SQL_SERVER_NAME) + // Retrieves a UTF-8 string with the native version of the Flight SQL Server. + SqlInfoFlightSqlServerVersion = SqlInfo(pb.SqlInfo_FLIGHT_SQL_SERVER_VERSION) + // Retrieves a UTF-8 string with the Arrow format version of the Flight SQL Server. + SqlInfoFlightSqlServerArrowVersion = SqlInfo(pb.SqlInfo_FLIGHT_SQL_SERVER_ARROW_VERSION) + + // Retrieves a boolean value indicating whether the Flight SQL Server is read only. + // + // Returns: + // - false: if read-write + // - true: if read only + SqlInfoFlightSqlServerReadOnly = SqlInfo(pb.SqlInfo_FLIGHT_SQL_SERVER_READ_ONLY) + + // SQL Syntax Information + // Values [500-1000): provide information about the supported SQL Syntax + + // Retrieves a boolean value indicating whether the Flight SQL Server supports CREATE and DROP of catalogs. + // + // Returns: + // - false: if it doesn't support CREATE and DROP of catalogs. + // - true: if it supports CREATE and DROP of catalogs. + SqlInfoDDLCatalog = SqlInfo(pb.SqlInfo_SQL_DDL_CATALOG) + + // Retrieves a boolean value indicating whether the Flight SQL Server supports CREATE and DROP of schemas. + // + // Returns: + // - false: if it doesn't support CREATE and DROP of schemas. + // - true: if it supports CREATE and DROP of schemas. + SqlInfoDDLSchema = SqlInfo(pb.SqlInfo_SQL_DDL_SCHEMA) + + // Indicates whether the Flight SQL Server supports CREATE and DROP of tables. + // + // Returns: + // - false: if it doesn't support CREATE and DROP of tables. + // - true: if it supports CREATE and DROP of tables. + SqlInfoDDLTable = SqlInfo(pb.SqlInfo_SQL_DDL_TABLE) + + // Retrieves a int32 ordinal representing the case sensitivity of catalog, table, schema and table names. + // + // The possible values are listed in `arrow.flight.protocol.sql.SqlSupportedCaseSensitivity`. + SqlInfoIdentifierCase = SqlInfo(pb.SqlInfo_SQL_IDENTIFIER_CASE) + // Retrieves a UTF-8 string with the supported character(s) used to surround a delimited identifier. + SqlInfoIdentifierQuoteChar = SqlInfo(pb.SqlInfo_SQL_IDENTIFIER_QUOTE_CHAR) + + // Retrieves a int32 describing the case sensitivity of quoted identifiers. + // + // The possible values are listed in `arrow.flight.protocol.sql.SqlSupportedCaseSensitivity`. + SqlInfoQuotedIdentifierCase = SqlInfo(pb.SqlInfo_SQL_QUOTED_IDENTIFIER_CASE) + + // Retrieves a boolean value indicating whether all tables are selectable. + // + // Returns: + // - false: if not all tables are selectable or if none are; + // - true: if all tables are selectable. + SqlInfoAllTablesAreASelectable = SqlInfo(pb.SqlInfo_SQL_ALL_TABLES_ARE_SELECTABLE) + + // Retrieves the null ordering. + // + // Returns a int32 ordinal for the null ordering being used, as described in + // `arrow.flight.protocol.sql.SqlNullOrdering`. + SqlInfoNullOrdering = SqlInfo(pb.SqlInfo_SQL_NULL_ORDERING) + // Retrieves a UTF-8 string list with values of the supported keywords. + SqlInfoKeywords = SqlInfo(pb.SqlInfo_SQL_KEYWORDS) + // Retrieves a UTF-8 string list with values of the supported numeric functions. + SqlInfoNumericFunctions = SqlInfo(pb.SqlInfo_SQL_NUMERIC_FUNCTIONS) + // Retrieves a UTF-8 string list with values of the supported string functions. + SqlInfoStringFunctions = SqlInfo(pb.SqlInfo_SQL_STRING_FUNCTIONS) + // Retrieves a UTF-8 string list with values of the supported system functions. + SqlInfoSystemFunctions = SqlInfo(pb.SqlInfo_SQL_SYSTEM_FUNCTIONS) + // Retrieves a UTF-8 string list with values of the supported datetime functions. + SqlInfoDateTimeFunctions = SqlInfo(pb.SqlInfo_SQL_DATETIME_FUNCTIONS) + + // Retrieves the UTF-8 string that can be used to escape wildcard characters. + // This is the string that can be used to escape '_' or '%' in the catalog search parameters that are a pattern + // (and therefore use one of the wildcard characters). + // The '_' character represents any single character; the '%' character represents any sequence of zero or more + // characters. + SqlInfoSearchStringEscape = SqlInfo(pb.SqlInfo_SQL_SEARCH_STRING_ESCAPE) + + // Retrieves a UTF-8 string with all the "extra" characters that can be used in unquoted identifier names + // (those beyond a-z, A-Z, 0-9 and _). + SqlInfoExtraNameChars = SqlInfo(pb.SqlInfo_SQL_EXTRA_NAME_CHARACTERS) + + // Retrieves a boolean value indicating whether column aliasing is supported. + // If so, the SQL AS clause can be used to provide names for computed columns or to provide alias names for columns + // as required. + // + // Returns: + // - false: if column aliasing is unsupported; + // - true: if column aliasing is supported. + SqlInfoSupportsColumnAliasing = SqlInfo(pb.SqlInfo_SQL_SUPPORTS_COLUMN_ALIASING) + + // Retrieves a boolean value indicating whether concatenations between null and non-null values being + // null are supported. + // + // - Returns: + // - false: if concatenations between null and non-null values being null are unsupported; + // - true: if concatenations between null and non-null values being null are supported. + SqlInfoNullPlusNullIsNull = SqlInfo(pb.SqlInfo_SQL_NULL_PLUS_NULL_IS_NULL) + + // Retrieves a map where the key is the type to convert from and the value is a list with the types to convert to, + // indicating the supported conversions. Each key and each item on the list value is a value to a predefined type on + // SqlSupportsConvert enum. + // The returned map will be: map> + SqlInfoSupportsConvert = SqlInfo(pb.SqlInfo_SQL_SUPPORTS_CONVERT) + + // Retrieves a boolean value indicating whether, when table correlation names are supported, + // they are restricted to being different from the names of the tables. + // + // Returns: + // - false: if table correlation names are unsupported; + // - true: if table correlation names are supported. + SqlInfoSupportsTableCorrelationNames = SqlInfo(pb.SqlInfo_SQL_SUPPORTS_TABLE_CORRELATION_NAMES) + + // Retrieves a boolean value indicating whether, when table correlation names are supported, + // they are restricted to being different from the names of the tables. + // + // Returns: + // - false: if different table correlation names are unsupported; + // - true: if different table correlation names are supported + SqlInfoSupportsDifferentTableCorrelationNames = SqlInfo(pb.SqlInfo_SQL_SUPPORTS_DIFFERENT_TABLE_CORRELATION_NAMES) + + // Retrieves a boolean value indicating whether expressions in ORDER BY lists are supported. + // + // Returns: + // - false: if expressions in ORDER BY are unsupported; + // - true: if expressions in ORDER BY are supported; + SqlInfoSupportsExpressionsInOrderBy = SqlInfo(pb.SqlInfo_SQL_SUPPORTS_EXPRESSIONS_IN_ORDER_BY) + + // Retrieves a boolean value indicating whether using a column that is not in the SELECT statement in a GROUP BY + // clause is supported. + // + // Returns: + // - false: if using a column that is not in the SELECT statement in a GROUP BY clause is unsupported; + // - true: if using a column that is not in the SELECT statement in a GROUP BY clause is supported. + SqlInfoSupportsOrderByUnrelated = SqlInfo(pb.SqlInfo_SQL_SUPPORTS_ORDER_BY_UNRELATED) + + // Retrieves the supported GROUP BY commands; + // + // Returns an int32 bitmask value representing the supported commands. + // The returned bitmask should be parsed in order to retrieve the supported commands. + // + // For instance: + // - return 0 (\b0) => [] (GROUP BY is unsupported); + // - return 1 (\b1) => [SQL_GROUP_BY_UNRELATED]; + // - return 2 (\b10) => [SQL_GROUP_BY_BEYOND_SELECT]; + // - return 3 (\b11) => [SQL_GROUP_BY_UNRELATED, SQL_GROUP_BY_BEYOND_SELECT]. + // Valid GROUP BY types are described under `arrow.flight.protocol.sql.SqlSupportedGroupBy`. + SqlInfoSupportedGroupBy = SqlInfo(pb.SqlInfo_SQL_SUPPORTED_GROUP_BY) + + // Retrieves a boolean value indicating whether specifying a LIKE escape clause is supported. + // + // Returns: + // - false: if specifying a LIKE escape clause is unsupported; + // - true: if specifying a LIKE escape clause is supported. + SqlInfoSupportsLikeEscapeClause = SqlInfo(pb.SqlInfo_SQL_SUPPORTS_LIKE_ESCAPE_CLAUSE) + + // Retrieves a boolean value indicating whether columns may be defined as non-nullable. + // + // Returns: + // - false: if columns cannot be defined as non-nullable; + // - true: if columns may be defined as non-nullable. + SqlInfoSupportsNonNullableColumns = SqlInfo(pb.SqlInfo_SQL_SUPPORTS_NON_NULLABLE_COLUMNS) + + // Retrieves the supported SQL grammar level as per the ODBC specification. + // + // Returns an int32 bitmask value representing the supported SQL grammar level. + // The returned bitmask should be parsed in order to retrieve the supported grammar levels. + // + // For instance: + // - return 0 (\b0) => [] (SQL grammar is unsupported); + // - return 1 (\b1) => [SQL_MINIMUM_GRAMMAR]; + // - return 2 (\b10) => [SQL_CORE_GRAMMAR]; + // - return 3 (\b11) => [SQL_MINIMUM_GRAMMAR, SQL_CORE_GRAMMAR]; + // - return 4 (\b100) => [SQL_EXTENDED_GRAMMAR]; + // - return 5 (\b101) => [SQL_MINIMUM_GRAMMAR, SQL_EXTENDED_GRAMMAR]; + // - return 6 (\b110) => [SQL_CORE_GRAMMAR, SQL_EXTENDED_GRAMMAR]; + // - return 7 (\b111) => [SQL_MINIMUM_GRAMMAR, SQL_CORE_GRAMMAR, SQL_EXTENDED_GRAMMAR]. + // Valid SQL grammar levels are described under `arrow.flight.protocol.sql.SupportedSqlGrammar`. + SqlInfoSupportedGrammar = SqlInfo(pb.SqlInfo_SQL_SUPPORTED_GRAMMAR) + + // Retrieves the supported ANSI92 SQL grammar level. + // + // Returns an int32 bitmask value representing the supported ANSI92 SQL grammar level. + // The returned bitmask should be parsed in order to retrieve the supported commands. + // + // For instance: + // - return 0 (\b0) => [] (ANSI92 SQL grammar is unsupported); + // - return 1 (\b1) => [ANSI92_ENTRY_SQL]; + // - return 2 (\b10) => [ANSI92_INTERMEDIATE_SQL]; + // - return 3 (\b11) => [ANSI92_ENTRY_SQL, ANSI92_INTERMEDIATE_SQL]; + // - return 4 (\b100) => [ANSI92_FULL_SQL]; + // - return 5 (\b101) => [ANSI92_ENTRY_SQL, ANSI92_FULL_SQL]; + // - return 6 (\b110) => [ANSI92_INTERMEDIATE_SQL, ANSI92_FULL_SQL]; + // - return 7 (\b111) => [ANSI92_ENTRY_SQL, ANSI92_INTERMEDIATE_SQL, ANSI92_FULL_SQL]. + // Valid ANSI92 SQL grammar levels are described under `arrow.flight.protocol.sql.SupportedAnsi92SqlGrammarLevel`. + SqlInfoANSI92SupportedLevel = SqlInfo(pb.SqlInfo_SQL_ANSI92_SUPPORTED_LEVEL) + + // Retrieves a boolean value indicating whether the SQL Integrity Enhancement Facility is supported. + // + // Returns: + // - false: if the SQL Integrity Enhancement Facility is supported; + // - true: if the SQL Integrity Enhancement Facility is supported. + SqlInfoSupportsIntegrityEnhancementFacility = SqlInfo(pb.SqlInfo_SQL_SUPPORTS_INTEGRITY_ENHANCEMENT_FACILITY) + + // Retrieves the support level for SQL OUTER JOINs. + // + // Returns a int32 ordinal for the SQL ordering being used, as described in + // `arrow.flight.protocol.sql.SqlOuterJoinsSupportLevel`. + SqlInfoOuterJoinsSupportLevel = SqlInfo(pb.SqlInfo_SQL_OUTER_JOINS_SUPPORT_LEVEL) + + // Retrieves a UTF-8 string with the preferred term for "schema". + SqlInfoSchemaTerm = SqlInfo(pb.SqlInfo_SQL_SCHEMA_TERM) + // Retrieves a UTF-8 string with the preferred term for "procedure". + SqlInfoProcedureTerm = SqlInfo(pb.SqlInfo_SQL_PROCEDURE_TERM) + + // Retrieves a UTF-8 string with the preferred term for "catalog". + // If a empty string is returned its assumed that the server does NOT supports catalogs. + SqlInfoCatalogTerm = SqlInfo(pb.SqlInfo_SQL_CATALOG_TERM) + + // Retrieves a boolean value indicating whether a catalog appears at the start of a fully qualified table name. + // + // - false: if a catalog does not appear at the start of a fully qualified table name; + // - true: if a catalog appears at the start of a fully qualified table name. + SqlInfoCatalogAtStart = SqlInfo(pb.SqlInfo_SQL_CATALOG_AT_START) + + // Retrieves the supported actions for a SQL schema. + // + // Returns an int32 bitmask value representing the supported actions for a SQL schema. + // The returned bitmask should be parsed in order to retrieve the supported actions for a SQL schema. + // + // For instance: + // - return 0 (\b0) => [] (no supported actions for SQL schema); + // - return 1 (\b1) => [SQL_ELEMENT_IN_PROCEDURE_CALLS]; + // - return 2 (\b10) => [SQL_ELEMENT_IN_INDEX_DEFINITIONS]; + // - return 3 (\b11) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_INDEX_DEFINITIONS]; + // - return 4 (\b100) => [SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]; + // - return 5 (\b101) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]; + // - return 6 (\b110) => [SQL_ELEMENT_IN_INDEX_DEFINITIONS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]; + // - return 7 (\b111) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_INDEX_DEFINITIONS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]. + // Valid actions for a SQL schema described under `arrow.flight.protocol.sql.SqlSupportedElementActions`. + SqlInfoSchemasSupportedActions = SqlInfo(pb.SqlInfo_SQL_SCHEMAS_SUPPORTED_ACTIONS) + + // Retrieves the supported actions for a SQL schema. + // + // Returns an int32 bitmask value representing the supported actions for a SQL catalog. + // The returned bitmask should be parsed in order to retrieve the supported actions for a SQL catalog. + // + // For instance: + // - return 0 (\b0) => [] (no supported actions for SQL catalog); + // - return 1 (\b1) => [SQL_ELEMENT_IN_PROCEDURE_CALLS]; + // - return 2 (\b10) => [SQL_ELEMENT_IN_INDEX_DEFINITIONS]; + // - return 3 (\b11) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_INDEX_DEFINITIONS]; + // - return 4 (\b100) => [SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]; + // - return 5 (\b101) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]; + // - return 6 (\b110) => [SQL_ELEMENT_IN_INDEX_DEFINITIONS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]; + // - return 7 (\b111) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_INDEX_DEFINITIONS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]. + // Valid actions for a SQL catalog are described under `arrow.flight.protocol.sql.SqlSupportedElementActions`. + SqlInfoCatalogsSupportedActions = SqlInfo(pb.SqlInfo_SQL_CATALOGS_SUPPORTED_ACTIONS) + + // Retrieves the supported SQL positioned commands. + // + // Returns an int32 bitmask value representing the supported SQL positioned commands. + // The returned bitmask should be parsed in order to retrieve the supported SQL positioned commands. + // + // For instance: + // - return 0 (\b0) => [] (no supported SQL positioned commands); + // - return 1 (\b1) => [SQL_POSITIONED_DELETE]; + // - return 2 (\b10) => [SQL_POSITIONED_UPDATE]; + // - return 3 (\b11) => [SQL_POSITIONED_DELETE, SQL_POSITIONED_UPDATE]. + // Valid SQL positioned commands are described under `arrow.flight.protocol.sql.SqlSupportedPositionedCommands`. + SqlInfoSupportedPositionedCommands = SqlInfo(pb.SqlInfo_SQL_SUPPORTED_POSITIONED_COMMANDS) + + // Retrieves a boolean value indicating whether SELECT FOR UPDATE statements are supported. + // + // Returns: + // - false: if SELECT FOR UPDATE statements are unsupported; + // - true: if SELECT FOR UPDATE statements are supported. + SqlInfoSelectForUpdateSupported = SqlInfo(pb.SqlInfo_SQL_SELECT_FOR_UPDATE_SUPPORTED) + + // Retrieves a boolean value indicating whether stored procedure calls that use the stored procedure escape syntax + // are supported. + // + // Returns: + // - false: if stored procedure calls that use the stored procedure escape syntax are unsupported; + // - true: if stored procedure calls that use the stored procedure escape syntax are supported. + SqlInfoStoredProceduresSupported = SqlInfo(pb.SqlInfo_SQL_STORED_PROCEDURES_SUPPORTED) + + // Retrieves the supported SQL subqueries. + // + // Returns an int32 bitmask value representing the supported SQL subqueries. + // The returned bitmask should be parsed in order to retrieve the supported SQL subqueries. + // + // For instance: + // - return 0 (\b0) => [] (no supported SQL subqueries); + // - return 1 (\b1) => [SQL_SUBQUERIES_IN_COMPARISONS]; + // - return 2 (\b10) => [SQL_SUBQUERIES_IN_EXISTS]; + // - return 3 (\b11) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_EXISTS]; + // - return 4 (\b100) => [SQL_SUBQUERIES_IN_INS]; + // - return 5 (\b101) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_INS]; + // - return 6 (\b110) => [SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_EXISTS]; + // - return 7 (\b111) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_INS]; + // - return 8 (\b1000) => [SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 9 (\b1001) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 10 (\b1010) => [SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 11 (\b1011) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 12 (\b1100) => [SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 13 (\b1101) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 14 (\b1110) => [SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 15 (\b1111) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - ... + // Valid SQL subqueries are described under `arrow.flight.protocol.sql.SqlSupportedSubqueries`. + SqlInfoSupportedSubqueries = SqlInfo(pb.SqlInfo_SQL_SUPPORTED_SUBQUERIES) + + // Retrieves a boolean value indicating whether correlated subqueries are supported. + // + // Returns: + // - false: if correlated subqueries are unsupported; + // - true: if correlated subqueries are supported. + SqlInfoCorrelatedSubqueriesSupported = SqlInfo(pb.SqlInfo_SQL_CORRELATED_SUBQUERIES_SUPPORTED) + + // Retrieves the supported SQL UNIONs. + // + // Returns an int32 bitmask value representing the supported SQL UNIONs. + // The returned bitmask should be parsed in order to retrieve the supported SQL UNIONs. + // + // For instance: + // - return 0 (\b0) => [] (no supported SQL positioned commands); + // - return 1 (\b1) => [SQL_UNION]; + // - return 2 (\b10) => [SQL_UNION_ALL]; + // - return 3 (\b11) => [SQL_UNION, SQL_UNION_ALL]. + // Valid SQL positioned commands are described under `arrow.flight.protocol.sql.SqlSupportedUnions`. + SqlInfoSupportedUnions = SqlInfo(pb.SqlSupportedUnions_SQL_UNION) + + // Retrieves a int64 value representing the maximum number of hex characters allowed in an inline binary literal. + SqlInfoMaxBinaryLiteralLen = SqlInfo(pb.SqlInfo_SQL_MAX_BINARY_LITERAL_LENGTH) + // Retrieves a int64 value representing the maximum number of characters allowed for a character literal. + SqlInfoMaxCharLiteralLen = SqlInfo(pb.SqlInfo_SQL_MAX_CHAR_LITERAL_LENGTH) + // Retrieves a int64 value representing the maximum number of characters allowed for a column name. + SqlInfoMaxColumnNameLen = SqlInfo(pb.SqlInfo_SQL_MAX_COLUMN_NAME_LENGTH) + // Retrieves a int64 value representing the the maximum number of columns allowed in a GROUP BY clause. + SqlInfoMaxColumnsInGroupBy = SqlInfo(pb.SqlInfo_SQL_MAX_COLUMNS_IN_GROUP_BY) + // Retrieves a int64 value representing the maximum number of columns allowed in an index. + SqlInfoMaxColumnsInIndex = SqlInfo(pb.SqlInfo_SQL_MAX_COLUMNS_IN_INDEX) + // Retrieves a int64 value representing the maximum number of columns allowed in an ORDER BY clause. + SqlInfoMaxColumnsInOrderBy = SqlInfo(pb.SqlInfo_SQL_MAX_COLUMNS_IN_ORDER_BY) + // Retrieves a int64 value representing the maximum number of columns allowed in a SELECT list. + SqlInfoMaxColumnsInSelect = SqlInfo(pb.SqlInfo_SQL_MAX_COLUMNS_IN_SELECT) + // Retrieves a int64 value representing the maximum number of columns allowed in a table. + SqlInfoMaxColumnsInTable = SqlInfo(pb.SqlInfo_SQL_MAX_COLUMNS_IN_TABLE) + // Retrieves a int64 value representing the maximum number of concurrent connections possible. + SqlInfoMaxConnections = SqlInfo(pb.SqlInfo_SQL_MAX_CONNECTIONS) + // Retrieves a int64 value the maximum number of characters allowed in a cursor name. + SqlInfoMaxCursorNameLen = SqlInfo(pb.SqlInfo_SQL_MAX_CURSOR_NAME_LENGTH) + + // Retrieves a int64 value representing the maximum number of bytes allowed for an index, + // including all of the parts of the index. + SqlInfoMaxIndexLen = SqlInfo(pb.SqlInfo_SQL_MAX_INDEX_LENGTH) + // Retrieves a int64 value representing the maximum number of characters allowed in a schema name. + SqlInfoDBSchemaNameLen = SqlInfo(pb.SqlInfo_SQL_DB_SCHEMA_NAME_LENGTH) + // Retrieves a int64 value representing the maximum number of characters allowed in a procedure name. + SqlInfoMaxProcedureNameLen = SqlInfo(pb.SqlInfo_SQL_MAX_PROCEDURE_NAME_LENGTH) + // Retrieves a int64 value representing the maximum number of characters allowed in a catalog name. + SqlInfoMaxCatalogNameLen = SqlInfo(pb.SqlInfo_SQL_MAX_CATALOG_NAME_LENGTH) + // Retrieves a int64 value representing the maximum number of bytes allowed in a single row. + SqlInfoMaxRowSize = SqlInfo(pb.SqlInfo_SQL_MAX_ROW_SIZE) + + // Retrieves a boolean indicating whether the return value for the JDBC method getMaxRowSize includes the SQL + // data types LONGVARCHAR and LONGVARBINARY. + // + // Returns: + // - false: if return value for the JDBC method getMaxRowSize does + // not include the SQL data types LONGVARCHAR and LONGVARBINARY; + // - true: if return value for the JDBC method getMaxRowSize includes + // the SQL data types LONGVARCHAR and LONGVARBINARY. + SqlInfoMaxRowSizeIncludesBlobs = SqlInfo(pb.SqlInfo_SQL_MAX_ROW_SIZE_INCLUDES_BLOBS) + + // Retrieves a int64 value representing the maximum number of characters allowed for an SQL statement; + // a result of 0 (zero) means that there is no limit or the limit is not known. + SqlInfoMaxStatementLen = SqlInfo(pb.SqlInfo_SQL_MAX_STATEMENT_LENGTH) + // Retrieves a int64 value representing the maximum number of active statements that can be open at the same time. + SqlInfoMaxStatements = SqlInfo(pb.SqlInfo_SQL_MAX_STATEMENTS) + // Retrieves a int64 value representing the maximum number of characters allowed in a table name. + SqlInfoMaxTableNameLen = SqlInfo(pb.SqlInfo_SQL_MAX_TABLE_NAME_LENGTH) + // Retrieves a int64 value representing the maximum number of tables allowed in a SELECT statement. + SqlInfoMaxTablesInSelect = SqlInfo(pb.SqlInfo_SQL_MAX_TABLES_IN_SELECT) + // Retrieves a int64 value representing the maximum number of characters allowed in a user name. + SqlInfoMaxUsernameLen = SqlInfo(pb.SqlInfo_SQL_MAX_USERNAME_LENGTH) + + // Retrieves this database's default transaction isolation level as described in + // `arrow.flight.protocol.sql.SqlTransactionIsolationLevel`. + // + // Returns a int32 ordinal for the SQL transaction isolation level. + SqlInfoDefaultTransactionIsolation = SqlInfo(pb.SqlInfo_SQL_DEFAULT_TRANSACTION_ISOLATION) + + // Retrieves a boolean value indicating whether transactions are supported. If not, invoking the method commit is a + // noop, and the isolation level is `arrow.flight.protocol.sql.SqlTransactionIsolationLevel.TRANSACTION_NONE`. + // + // Returns: + // - false: if transactions are unsupported; + // - true: if transactions are supported. + SqlInfoTransactionsSupported = SqlInfo(pb.SqlInfo_SQL_TRANSACTIONS_SUPPORTED) + + // Retrieves the supported transactions isolation levels. + // + // Returns an int32 bitmask value representing the supported transactions isolation levels. + // The returned bitmask should be parsed in order to retrieve the supported transactions isolation levels. + // + // For instance: + // - return 0 (\b0) => [] (no supported SQL transactions isolation levels); + // - return 1 (\b1) => [SQL_TRANSACTION_NONE]; + // - return 2 (\b10) => [SQL_TRANSACTION_READ_UNCOMMITTED]; + // - return 3 (\b11) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_READ_UNCOMMITTED]; + // - return 4 (\b100) => [SQL_TRANSACTION_REPEATABLE_READ]; + // - return 5 (\b101) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 6 (\b110) => [SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 7 (\b111) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 8 (\b1000) => [SQL_TRANSACTION_REPEATABLE_READ]; + // - return 9 (\b1001) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 10 (\b1010) => [SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 11 (\b1011) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 12 (\b1100) => [SQL_TRANSACTION_REPEATABLE_READ, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 13 (\b1101) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_REPEATABLE_READ, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 14 (\b1110) => [SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 15 (\b1111) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 16 (\b10000) => [SQL_TRANSACTION_SERIALIZABLE]; + // - ... + // Valid SQL positioned commands are described under `arrow.flight.protocol.sql.SqlTransactionIsolationLevel`. + SqlInfoSupportedTransactionsIsolationlevels = SqlInfo(pb.SqlInfo_SQL_SUPPORTED_TRANSACTIONS_ISOLATION_LEVELS) + + // Retrieves a boolean value indicating whether a data definition statement within a transaction forces + // the transaction to commit. + // + // Returns: + // - false: if a data definition statement within a transaction does not force the transaction to commit; + // - true: if a data definition statement within a transaction forces the transaction to commit. + SqlInfoDataDefinitionCausesTransactionCommit = SqlInfo(pb.SqlInfo_SQL_DATA_DEFINITION_CAUSES_TRANSACTION_COMMIT) + + // Retrieves a boolean value indicating whether a data definition statement within a transaction is ignored. + // + // Returns: + // - false: if a data definition statement within a transaction is taken into account; + // - true: a data definition statement within a transaction is ignored. + SqlInfoDataDefinitionsInTransactionsIgnored = SqlInfo(pb.SqlInfo_SQL_DATA_DEFINITIONS_IN_TRANSACTIONS_IGNORED) + + // Retrieves an int32 bitmask value representing the supported result set types. + // The returned bitmask should be parsed in order to retrieve the supported result set types. + // + // For instance: + // - return 0 (\b0) => [] (no supported result set types); + // - return 1 (\b1) => [SQL_RESULT_SET_TYPE_UNSPECIFIED]; + // - return 2 (\b10) => [SQL_RESULT_SET_TYPE_FORWARD_ONLY]; + // - return 3 (\b11) => [SQL_RESULT_SET_TYPE_UNSPECIFIED, SQL_RESULT_SET_TYPE_FORWARD_ONLY]; + // - return 4 (\b100) => [SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE]; + // - return 5 (\b101) => [SQL_RESULT_SET_TYPE_UNSPECIFIED, SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE]; + // - return 6 (\b110) => [SQL_RESULT_SET_TYPE_FORWARD_ONLY, SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE]; + // - return 7 (\b111) => [SQL_RESULT_SET_TYPE_UNSPECIFIED, SQL_RESULT_SET_TYPE_FORWARD_ONLY, SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE]; + // - return 8 (\b1000) => [SQL_RESULT_SET_TYPE_SCROLL_SENSITIVE]; + // - ... + // Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetType`. + SqlInfoSupportedResultSetTypes = SqlInfo(pb.SqlInfo_SQL_SUPPORTED_RESULT_SET_TYPES) + + // Returns an int32 bitmask value concurrency types supported for + // `arrow.flight.protocol.sql.SqlSupportedResultSetType.SQL_RESULT_SET_TYPE_UNSPECIFIED`. + // + // For instance: + // - return 0 (\b0) => [] (no supported concurrency types for this result set type) + // - return 1 (\b1) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED] + // - return 2 (\b10) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 3 (\b11) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 4 (\b100) => [SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 5 (\b101) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 6 (\b110) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 7 (\b111) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency`. + SqlInfoSupportedConcurrenciesForResultSetUnspecified = SqlInfo(pb.SqlInfo_SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_UNSPECIFIED) + + // Returns an int32 bitmask value concurrency types supported for + // `arrow.flight.protocol.sql.SqlSupportedResultSetType.SQL_RESULT_SET_TYPE_FORWARD_ONLY`. + // + // For instance: + // - return 0 (\b0) => [] (no supported concurrency types for this result set type) + // - return 1 (\b1) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED] + // - return 2 (\b10) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 3 (\b11) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 4 (\b100) => [SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 5 (\b101) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 6 (\b110) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 7 (\b111) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency`. + SqlInfoSupportedConcurrenciesForResultSetForwardOnly = SqlInfo(pb.SqlInfo_SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_FORWARD_ONLY) + + // Returns an int32 bitmask value concurrency types supported for + // `arrow.flight.protocol.sql.SqlSupportedResultSetType.SQL_RESULT_SET_TYPE_SCROLL_SENSITIVE`. + // + // For instance: + // - return 0 (\b0) => [] (no supported concurrency types for this result set type) + // - return 1 (\b1) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED] + // - return 2 (\b10) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 3 (\b11) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 4 (\b100) => [SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 5 (\b101) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 6 (\b110) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 7 (\b111) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency`. + SqlInfoSupportedConcurrenciesForResultSetScrollSensitive = SqlInfo(pb.SqlInfo_SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_SCROLL_SENSITIVE) + + // Returns an int32 bitmask value concurrency types supported for + // `arrow.flight.protocol.sql.SqlSupportedResultSetType.SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE`. + // + // For instance: + // - return 0 (\b0) => [] (no supported concurrency types for this result set type) + // - return 1 (\b1) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED] + // - return 2 (\b10) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 3 (\b11) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 4 (\b100) => [SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 5 (\b101) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 6 (\b110) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 7 (\b111) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency`. + SqlInfoSupportedConcurrenciesForResultSetScrollInensitive = SqlInfo(pb.SqlInfo_SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_SCROLL_INSENSITIVE) + + // Retrieves a boolean value indicating whether this database supports batch updates. + // + // - false: if this database does not support batch updates; + // - true: if this database supports batch updates. + SqlInfoBatchUpdatesSupported = SqlInfo(pb.SqlInfo_SQL_BATCH_UPDATES_SUPPORTED) + + // Retrieves a boolean value indicating whether this database supports savepoints. + // + // Returns: + // - false: if this database does not support savepoints; + // - true: if this database supports savepoints. + SqlInfoSavePointsSupported = SqlInfo(pb.SqlInfo_SQL_SAVEPOINTS_SUPPORTED) + + // Retrieves a boolean value indicating whether named parameters are supported in callable statements. + // + // Returns: + // - false: if named parameters in callable statements are unsupported; + // - true: if named parameters in callable statements are supported. + SqlInfoNamedParametersSupported = SqlInfo(pb.SqlInfo_SQL_NAMED_PARAMETERS_SUPPORTED) + + // Retrieves a boolean value indicating whether updates made to a LOB are made on a copy or directly to the LOB. + // + // Returns: + // - false: if updates made to a LOB are made directly to the LOB; + // - true: if updates made to a LOB are made on a copy. + SqlInfoLocatorsUpdateCopy = SqlInfo(pb.SqlInfo_SQL_LOCATORS_UPDATE_COPY) + + // Retrieves a boolean value indicating whether invoking user-defined or vendor functions + // using the stored procedure escape syntax is supported. + // + // Returns: + // - false: if invoking user-defined or vendor functions using the stored procedure escape syntax is unsupported; + // - true: if invoking user-defined or vendor functions using the stored procedure escape syntax is supported. + SqlInfoStoredFunctionsUsingCallSyntaxSupported = SqlInfo(pb.SqlInfo_SQL_STORED_FUNCTIONS_USING_CALL_SYNTAX_SUPPORTED) +) + +func (s SqlInfo) String() string { return pb.SqlInfo(int32(s)).String() } + +// SqlSupportedCaseSensitivity indicates whether something +// (e.g. an identifier) is case-sensitive +// +// duplicated from protobuf to avoid relying directly on the protobuf +// generated code, also making them shorter and easier to use +type SqlSupportedCaseSensitivity = pb.SqlSupportedCaseSensitivity + +const ( + SqlCaseSensitivityUnknown = pb.SqlSupportedCaseSensitivity_SQL_CASE_SENSITIVITY_UNKNOWN + SqlCaseSensitivityCaseInsensitive = pb.SqlSupportedCaseSensitivity_SQL_CASE_SENSITIVITY_CASE_INSENSITIVE + SqlCaseSensitivityUpperCase = pb.SqlSupportedCaseSensitivity_SQL_CASE_SENSITIVITY_UPPERCASE + SqlCaseSensitivityLowerCase = pb.SqlSupportedCaseSensitivity_SQL_CASE_SENSITIVITY_LOWERCASE +) + +// SqlNullOrdering indicates how nulls are sorted +// +// duplicated from protobuf to avoid relying directly on the protobuf +// generated code, also making them shorter and easier to use +type SqlNullOrdering = pb.SqlNullOrdering + +const ( + SqlNullOrderingSortHigh = pb.SqlNullOrdering_SQL_NULLS_SORTED_HIGH + SqlNullOrderingSortLow = pb.SqlNullOrdering_SQL_NULLS_SORTED_LOW + SqlNullOrderingSortAtStart = pb.SqlNullOrdering_SQL_NULLS_SORTED_AT_START + SqlNullOrderingSortAtEnd = pb.SqlNullOrdering_SQL_NULLS_SORTED_AT_END +) + +// SqlSupportsConvert indicates support for converting between different +// types. +// +// duplicated from protobuf to avoid relying directly on the protobuf +// generated code, also making them shorter and easier to use +type SqlSupportsConvert = pb.SqlSupportsConvert + +const ( + SqlConvertBigInt = pb.SqlSupportsConvert_SQL_CONVERT_BIGINT + SqlConvertBinary = pb.SqlSupportsConvert_SQL_CONVERT_BINARY + SqlConvertBit = pb.SqlSupportsConvert_SQL_CONVERT_BIT + SqlConvertChar = pb.SqlSupportsConvert_SQL_CONVERT_CHAR + SqlConvertDate = pb.SqlSupportsConvert_SQL_CONVERT_DATE + SqlConvertDecimal = pb.SqlSupportsConvert_SQL_CONVERT_DECIMAL + SqlConvertFloat = pb.SqlSupportsConvert_SQL_CONVERT_FLOAT + SqlConvertInteger = pb.SqlSupportsConvert_SQL_CONVERT_INTEGER + SqlConvertIntervalDayTime = pb.SqlSupportsConvert_SQL_CONVERT_INTERVAL_DAY_TIME + SqlConvertIntervalYearMonth = pb.SqlSupportsConvert_SQL_CONVERT_INTERVAL_YEAR_MONTH + SqlConvertLongVarbinary = pb.SqlSupportsConvert_SQL_CONVERT_LONGVARBINARY + SqlConvertLongVarchar = pb.SqlSupportsConvert_SQL_CONVERT_LONGVARCHAR + SqlConvertNumeric = pb.SqlSupportsConvert_SQL_CONVERT_NUMERIC + SqlConvertReal = pb.SqlSupportsConvert_SQL_CONVERT_REAL + SqlConvertSmallInt = pb.SqlSupportsConvert_SQL_CONVERT_SMALLINT + SqlConvertTime = pb.SqlSupportsConvert_SQL_CONVERT_TIME + SqlConvertTimestamp = pb.SqlSupportsConvert_SQL_CONVERT_TIMESTAMP + SqlConvertTinyInt = pb.SqlSupportsConvert_SQL_CONVERT_TINYINT + SqlConvertVarbinary = pb.SqlSupportsConvert_SQL_CONVERT_VARBINARY + SqlConvertVarchar = pb.SqlSupportsConvert_SQL_CONVERT_VARCHAR +) diff --git a/go/arrow/flight/gen.go b/go/arrow/flight/gen.go index be55119c7fd20..4109059af8ed1 100644 --- a/go/arrow/flight/gen.go +++ b/go/arrow/flight/gen.go @@ -17,3 +17,4 @@ package flight //go:generate protoc -I../../../format --go_out=./internal/flight --go-grpc_out=./internal/flight --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative Flight.proto +//go:generate protoc --experimental_allow_proto3_optional -I../../../format --go_out=./internal/flight --go-grpc_out=./internal/flight --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative FlightSql.proto diff --git a/go/arrow/flight/internal/flight/Flight.pb.go b/go/arrow/flight/internal/flight/Flight.pb.go index 18ba75bb66ec6..b7be492acd424 100644 --- a/go/arrow/flight/internal/flight/Flight.pb.go +++ b/go/arrow/flight/internal/flight/Flight.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 +// protoc-gen-go v1.28.1 // protoc v3.12.4 // source: Flight.proto @@ -666,8 +666,15 @@ type FlightInfo struct { // The descriptor associated with this info. FlightDescriptor *FlightDescriptor `protobuf:"bytes,2,opt,name=flight_descriptor,json=flightDescriptor,proto3" json:"flight_descriptor,omitempty"` // - // A list of endpoints associated with the flight. To consume the whole - // flight, all endpoints must be consumed. + // A list of endpoints associated with the flight. To consume the + // whole flight, all endpoints (and hence all Tickets) must be + // consumed. Endpoints can be consumed in any order. + // + // In other words, an application can use multiple endpoints to + // represent partitioned data. + // + // There is no ordering defined on endpoints. Hence, if the returned + // data has an ordering, it should be returned in a single endpoint. Endpoint []*FlightEndpoint `protobuf:"bytes,3,rep,name=endpoint,proto3" json:"endpoint,omitempty"` // Set these to -1 if unknown. TotalRecords int64 `protobuf:"varint,4,opt,name=total_records,json=totalRecords,proto3" json:"total_records,omitempty"` @@ -752,9 +759,20 @@ type FlightEndpoint struct { // Token used to retrieve this stream. Ticket *Ticket `protobuf:"bytes,1,opt,name=ticket,proto3" json:"ticket,omitempty"` // - // A list of URIs where this ticket can be redeemed. If the list is - // empty, the expectation is that the ticket can only be redeemed on the - // current service where the ticket was generated. + // A list of URIs where this ticket can be redeemed via DoGet(). + // + // If the list is empty, the expectation is that the ticket can only + // 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. + // + // In other words, an application can use multiple locations to + // represent redundant and/or load balanced services. Location []*Location `protobuf:"bytes,2,rep,name=location,proto3" json:"location,omitempty"` } @@ -857,6 +875,9 @@ func (x *Location) GetUri() string { // // An opaque identifier that the service can use to retrieve a particular // portion of a stream. +// +// Tickets are meant to be single use. It is an error/application-defined +// behavior to reuse a ticket. type Ticket struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1180,14 +1201,15 @@ var file_Flight_proto_rawDesc = []byte{ 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x21, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, - 0x65, 0x22, 0x00, 0x30, 0x01, 0x42, 0x67, 0x0a, 0x1c, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x65, 0x22, 0x00, 0x30, 0x01, 0x42, 0x76, 0x0a, 0x1c, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, - 0x2e, 0x69, 0x6d, 0x70, 0x6c, 0x5a, 0x28, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, + 0x2e, 0x69, 0x6d, 0x70, 0x6c, 0x5a, 0x37, 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, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x3b, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0xaa, - 0x02, 0x1c, 0x41, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x41, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x46, - 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x62, 0x06, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x6f, 0x2f, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2f, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2f, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0xaa, 0x02, + 0x1c, 0x41, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x41, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x46, 0x6c, + 0x69, 0x67, 0x68, 0x74, 0x2e, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/go/arrow/flight/internal/flight/FlightSql.pb.go b/go/arrow/flight/internal/flight/FlightSql.pb.go new file mode 100644 index 0000000000000..ca38190efbfe4 --- /dev/null +++ b/go/arrow/flight/internal/flight/FlightSql.pb.go @@ -0,0 +1,4396 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +//

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

+// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.28.1 +// protoc v3.12.4 +// source: FlightSql.proto + +package flight + +import ( + descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// Options for CommandGetSqlInfo. +type SqlInfo int32 + +const ( + // Retrieves a UTF-8 string with the name of the Flight SQL Server. + SqlInfo_FLIGHT_SQL_SERVER_NAME SqlInfo = 0 + // Retrieves a UTF-8 string with the native version of the Flight SQL Server. + SqlInfo_FLIGHT_SQL_SERVER_VERSION SqlInfo = 1 + // Retrieves a UTF-8 string with the Arrow format version of the Flight SQL Server. + SqlInfo_FLIGHT_SQL_SERVER_ARROW_VERSION SqlInfo = 2 + // + // Retrieves a boolean value indicating whether the Flight SQL Server is read only. + // + // Returns: + // - false: if read-write + // - true: if read only + SqlInfo_FLIGHT_SQL_SERVER_READ_ONLY SqlInfo = 3 + // + // Retrieves a boolean value indicating whether the Flight SQL Server supports CREATE and DROP of catalogs. + // + // Returns: + // - false: if it doesn't support CREATE and DROP of catalogs. + // - true: if it supports CREATE and DROP of catalogs. + SqlInfo_SQL_DDL_CATALOG SqlInfo = 500 + // + // Retrieves a boolean value indicating whether the Flight SQL Server supports CREATE and DROP of schemas. + // + // Returns: + // - false: if it doesn't support CREATE and DROP of schemas. + // - true: if it supports CREATE and DROP of schemas. + SqlInfo_SQL_DDL_SCHEMA SqlInfo = 501 + // + // Indicates whether the Flight SQL Server supports CREATE and DROP of tables. + // + // Returns: + // - false: if it doesn't support CREATE and DROP of tables. + // - true: if it supports CREATE and DROP of tables. + SqlInfo_SQL_DDL_TABLE SqlInfo = 502 + // + // Retrieves a int32 ordinal representing the case sensitivity of catalog, table, schema and table names. + // + // The possible values are listed in `arrow.flight.protocol.sql.SqlSupportedCaseSensitivity`. + SqlInfo_SQL_IDENTIFIER_CASE SqlInfo = 503 + // Retrieves a UTF-8 string with the supported character(s) used to surround a delimited identifier. + SqlInfo_SQL_IDENTIFIER_QUOTE_CHAR SqlInfo = 504 + // + // Retrieves a int32 describing the case sensitivity of quoted identifiers. + // + // The possible values are listed in `arrow.flight.protocol.sql.SqlSupportedCaseSensitivity`. + SqlInfo_SQL_QUOTED_IDENTIFIER_CASE SqlInfo = 505 + // + // Retrieves a boolean value indicating whether all tables are selectable. + // + // Returns: + // - false: if not all tables are selectable or if none are; + // - true: if all tables are selectable. + SqlInfo_SQL_ALL_TABLES_ARE_SELECTABLE SqlInfo = 506 + // + // Retrieves the null ordering. + // + // Returns a int32 ordinal for the null ordering being used, as described in + // `arrow.flight.protocol.sql.SqlNullOrdering`. + SqlInfo_SQL_NULL_ORDERING SqlInfo = 507 + // Retrieves a UTF-8 string list with values of the supported keywords. + SqlInfo_SQL_KEYWORDS SqlInfo = 508 + // Retrieves a UTF-8 string list with values of the supported numeric functions. + SqlInfo_SQL_NUMERIC_FUNCTIONS SqlInfo = 509 + // Retrieves a UTF-8 string list with values of the supported string functions. + SqlInfo_SQL_STRING_FUNCTIONS SqlInfo = 510 + // Retrieves a UTF-8 string list with values of the supported system functions. + SqlInfo_SQL_SYSTEM_FUNCTIONS SqlInfo = 511 + // Retrieves a UTF-8 string list with values of the supported datetime functions. + SqlInfo_SQL_DATETIME_FUNCTIONS SqlInfo = 512 + // + // Retrieves the UTF-8 string that can be used to escape wildcard characters. + // This is the string that can be used to escape '_' or '%' in the catalog search parameters that are a pattern + // (and therefore use one of the wildcard characters). + // The '_' character represents any single character; the '%' character represents any sequence of zero or more + // characters. + SqlInfo_SQL_SEARCH_STRING_ESCAPE SqlInfo = 513 + // + // Retrieves a UTF-8 string with all the "extra" characters that can be used in unquoted identifier names + // (those beyond a-z, A-Z, 0-9 and _). + SqlInfo_SQL_EXTRA_NAME_CHARACTERS SqlInfo = 514 + // + // Retrieves a boolean value indicating whether column aliasing is supported. + // If so, the SQL AS clause can be used to provide names for computed columns or to provide alias names for columns + // as required. + // + // Returns: + // - false: if column aliasing is unsupported; + // - true: if column aliasing is supported. + SqlInfo_SQL_SUPPORTS_COLUMN_ALIASING SqlInfo = 515 + // + // Retrieves a boolean value indicating whether concatenations between null and non-null values being + // null are supported. + // + // - Returns: + // - false: if concatenations between null and non-null values being null are unsupported; + // - true: if concatenations between null and non-null values being null are supported. + SqlInfo_SQL_NULL_PLUS_NULL_IS_NULL SqlInfo = 516 + // + // Retrieves a map where the key is the type to convert from and the value is a list with the types to convert to, + // indicating the supported conversions. Each key and each item on the list value is a value to a predefined type on + // SqlSupportsConvert enum. + // The returned map will be: map> + SqlInfo_SQL_SUPPORTS_CONVERT SqlInfo = 517 + // + // Retrieves a boolean value indicating whether, when table correlation names are supported, + // they are restricted to being different from the names of the tables. + // + // Returns: + // - false: if table correlation names are unsupported; + // - true: if table correlation names are supported. + SqlInfo_SQL_SUPPORTS_TABLE_CORRELATION_NAMES SqlInfo = 518 + // + // Retrieves a boolean value indicating whether, when table correlation names are supported, + // they are restricted to being different from the names of the tables. + // + // Returns: + // - false: if different table correlation names are unsupported; + // - true: if different table correlation names are supported + SqlInfo_SQL_SUPPORTS_DIFFERENT_TABLE_CORRELATION_NAMES SqlInfo = 519 + // + // Retrieves a boolean value indicating whether expressions in ORDER BY lists are supported. + // + // Returns: + // - false: if expressions in ORDER BY are unsupported; + // - true: if expressions in ORDER BY are supported; + SqlInfo_SQL_SUPPORTS_EXPRESSIONS_IN_ORDER_BY SqlInfo = 520 + // + // Retrieves a boolean value indicating whether using a column that is not in the SELECT statement in a GROUP BY + // clause is supported. + // + // Returns: + // - false: if using a column that is not in the SELECT statement in a GROUP BY clause is unsupported; + // - true: if using a column that is not in the SELECT statement in a GROUP BY clause is supported. + SqlInfo_SQL_SUPPORTS_ORDER_BY_UNRELATED SqlInfo = 521 + // + // Retrieves the supported GROUP BY commands; + // + // Returns an int32 bitmask value representing the supported commands. + // The returned bitmask should be parsed in order to retrieve the supported commands. + // + // For instance: + // - return 0 (\b0) => [] (GROUP BY is unsupported); + // - return 1 (\b1) => [SQL_GROUP_BY_UNRELATED]; + // - return 2 (\b10) => [SQL_GROUP_BY_BEYOND_SELECT]; + // - return 3 (\b11) => [SQL_GROUP_BY_UNRELATED, SQL_GROUP_BY_BEYOND_SELECT]. + // Valid GROUP BY types are described under `arrow.flight.protocol.sql.SqlSupportedGroupBy`. + SqlInfo_SQL_SUPPORTED_GROUP_BY SqlInfo = 522 + // + // Retrieves a boolean value indicating whether specifying a LIKE escape clause is supported. + // + // Returns: + // - false: if specifying a LIKE escape clause is unsupported; + // - true: if specifying a LIKE escape clause is supported. + SqlInfo_SQL_SUPPORTS_LIKE_ESCAPE_CLAUSE SqlInfo = 523 + // + // Retrieves a boolean value indicating whether columns may be defined as non-nullable. + // + // Returns: + // - false: if columns cannot be defined as non-nullable; + // - true: if columns may be defined as non-nullable. + SqlInfo_SQL_SUPPORTS_NON_NULLABLE_COLUMNS SqlInfo = 524 + // + // Retrieves the supported SQL grammar level as per the ODBC specification. + // + // Returns an int32 bitmask value representing the supported SQL grammar level. + // The returned bitmask should be parsed in order to retrieve the supported grammar levels. + // + // For instance: + // - return 0 (\b0) => [] (SQL grammar is unsupported); + // - return 1 (\b1) => [SQL_MINIMUM_GRAMMAR]; + // - return 2 (\b10) => [SQL_CORE_GRAMMAR]; + // - return 3 (\b11) => [SQL_MINIMUM_GRAMMAR, SQL_CORE_GRAMMAR]; + // - return 4 (\b100) => [SQL_EXTENDED_GRAMMAR]; + // - return 5 (\b101) => [SQL_MINIMUM_GRAMMAR, SQL_EXTENDED_GRAMMAR]; + // - return 6 (\b110) => [SQL_CORE_GRAMMAR, SQL_EXTENDED_GRAMMAR]; + // - return 7 (\b111) => [SQL_MINIMUM_GRAMMAR, SQL_CORE_GRAMMAR, SQL_EXTENDED_GRAMMAR]. + // Valid SQL grammar levels are described under `arrow.flight.protocol.sql.SupportedSqlGrammar`. + SqlInfo_SQL_SUPPORTED_GRAMMAR SqlInfo = 525 + // + // Retrieves the supported ANSI92 SQL grammar level. + // + // Returns an int32 bitmask value representing the supported ANSI92 SQL grammar level. + // The returned bitmask should be parsed in order to retrieve the supported commands. + // + // For instance: + // - return 0 (\b0) => [] (ANSI92 SQL grammar is unsupported); + // - return 1 (\b1) => [ANSI92_ENTRY_SQL]; + // - return 2 (\b10) => [ANSI92_INTERMEDIATE_SQL]; + // - return 3 (\b11) => [ANSI92_ENTRY_SQL, ANSI92_INTERMEDIATE_SQL]; + // - return 4 (\b100) => [ANSI92_FULL_SQL]; + // - return 5 (\b101) => [ANSI92_ENTRY_SQL, ANSI92_FULL_SQL]; + // - return 6 (\b110) => [ANSI92_INTERMEDIATE_SQL, ANSI92_FULL_SQL]; + // - return 7 (\b111) => [ANSI92_ENTRY_SQL, ANSI92_INTERMEDIATE_SQL, ANSI92_FULL_SQL]. + // Valid ANSI92 SQL grammar levels are described under `arrow.flight.protocol.sql.SupportedAnsi92SqlGrammarLevel`. + SqlInfo_SQL_ANSI92_SUPPORTED_LEVEL SqlInfo = 526 + // + // Retrieves a boolean value indicating whether the SQL Integrity Enhancement Facility is supported. + // + // Returns: + // - false: if the SQL Integrity Enhancement Facility is supported; + // - true: if the SQL Integrity Enhancement Facility is supported. + SqlInfo_SQL_SUPPORTS_INTEGRITY_ENHANCEMENT_FACILITY SqlInfo = 527 + // + // Retrieves the support level for SQL OUTER JOINs. + // + // Returns a int32 ordinal for the SQL ordering being used, as described in + // `arrow.flight.protocol.sql.SqlOuterJoinsSupportLevel`. + SqlInfo_SQL_OUTER_JOINS_SUPPORT_LEVEL SqlInfo = 528 + // Retrieves a UTF-8 string with the preferred term for "schema". + SqlInfo_SQL_SCHEMA_TERM SqlInfo = 529 + // Retrieves a UTF-8 string with the preferred term for "procedure". + SqlInfo_SQL_PROCEDURE_TERM SqlInfo = 530 + // + // Retrieves a UTF-8 string with the preferred term for "catalog". + // If a empty string is returned its assumed that the server does NOT supports catalogs. + SqlInfo_SQL_CATALOG_TERM SqlInfo = 531 + // + // Retrieves a boolean value indicating whether a catalog appears at the start of a fully qualified table name. + // + // - false: if a catalog does not appear at the start of a fully qualified table name; + // - true: if a catalog appears at the start of a fully qualified table name. + SqlInfo_SQL_CATALOG_AT_START SqlInfo = 532 + // + // Retrieves the supported actions for a SQL schema. + // + // Returns an int32 bitmask value representing the supported actions for a SQL schema. + // The returned bitmask should be parsed in order to retrieve the supported actions for a SQL schema. + // + // For instance: + // - return 0 (\b0) => [] (no supported actions for SQL schema); + // - return 1 (\b1) => [SQL_ELEMENT_IN_PROCEDURE_CALLS]; + // - return 2 (\b10) => [SQL_ELEMENT_IN_INDEX_DEFINITIONS]; + // - return 3 (\b11) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_INDEX_DEFINITIONS]; + // - return 4 (\b100) => [SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]; + // - return 5 (\b101) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]; + // - return 6 (\b110) => [SQL_ELEMENT_IN_INDEX_DEFINITIONS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]; + // - return 7 (\b111) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_INDEX_DEFINITIONS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]. + // Valid actions for a SQL schema described under `arrow.flight.protocol.sql.SqlSupportedElementActions`. + SqlInfo_SQL_SCHEMAS_SUPPORTED_ACTIONS SqlInfo = 533 + // + // Retrieves the supported actions for a SQL schema. + // + // Returns an int32 bitmask value representing the supported actions for a SQL catalog. + // The returned bitmask should be parsed in order to retrieve the supported actions for a SQL catalog. + // + // For instance: + // - return 0 (\b0) => [] (no supported actions for SQL catalog); + // - return 1 (\b1) => [SQL_ELEMENT_IN_PROCEDURE_CALLS]; + // - return 2 (\b10) => [SQL_ELEMENT_IN_INDEX_DEFINITIONS]; + // - return 3 (\b11) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_INDEX_DEFINITIONS]; + // - return 4 (\b100) => [SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]; + // - return 5 (\b101) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]; + // - return 6 (\b110) => [SQL_ELEMENT_IN_INDEX_DEFINITIONS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]; + // - return 7 (\b111) => [SQL_ELEMENT_IN_PROCEDURE_CALLS, SQL_ELEMENT_IN_INDEX_DEFINITIONS, SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS]. + // Valid actions for a SQL catalog are described under `arrow.flight.protocol.sql.SqlSupportedElementActions`. + SqlInfo_SQL_CATALOGS_SUPPORTED_ACTIONS SqlInfo = 534 + // + // Retrieves the supported SQL positioned commands. + // + // Returns an int32 bitmask value representing the supported SQL positioned commands. + // The returned bitmask should be parsed in order to retrieve the supported SQL positioned commands. + // + // For instance: + // - return 0 (\b0) => [] (no supported SQL positioned commands); + // - return 1 (\b1) => [SQL_POSITIONED_DELETE]; + // - return 2 (\b10) => [SQL_POSITIONED_UPDATE]; + // - return 3 (\b11) => [SQL_POSITIONED_DELETE, SQL_POSITIONED_UPDATE]. + // Valid SQL positioned commands are described under `arrow.flight.protocol.sql.SqlSupportedPositionedCommands`. + SqlInfo_SQL_SUPPORTED_POSITIONED_COMMANDS SqlInfo = 535 + // + // Retrieves a boolean value indicating whether SELECT FOR UPDATE statements are supported. + // + // Returns: + // - false: if SELECT FOR UPDATE statements are unsupported; + // - true: if SELECT FOR UPDATE statements are supported. + SqlInfo_SQL_SELECT_FOR_UPDATE_SUPPORTED SqlInfo = 536 + // + // Retrieves a boolean value indicating whether stored procedure calls that use the stored procedure escape syntax + // are supported. + // + // Returns: + // - false: if stored procedure calls that use the stored procedure escape syntax are unsupported; + // - true: if stored procedure calls that use the stored procedure escape syntax are supported. + SqlInfo_SQL_STORED_PROCEDURES_SUPPORTED SqlInfo = 537 + // + // Retrieves the supported SQL subqueries. + // + // Returns an int32 bitmask value representing the supported SQL subqueries. + // The returned bitmask should be parsed in order to retrieve the supported SQL subqueries. + // + // For instance: + // - return 0 (\b0) => [] (no supported SQL subqueries); + // - return 1 (\b1) => [SQL_SUBQUERIES_IN_COMPARISONS]; + // - return 2 (\b10) => [SQL_SUBQUERIES_IN_EXISTS]; + // - return 3 (\b11) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_EXISTS]; + // - return 4 (\b100) => [SQL_SUBQUERIES_IN_INS]; + // - return 5 (\b101) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_INS]; + // - return 6 (\b110) => [SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_EXISTS]; + // - return 7 (\b111) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_INS]; + // - return 8 (\b1000) => [SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 9 (\b1001) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 10 (\b1010) => [SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 11 (\b1011) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 12 (\b1100) => [SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 13 (\b1101) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 14 (\b1110) => [SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - return 15 (\b1111) => [SQL_SUBQUERIES_IN_COMPARISONS, SQL_SUBQUERIES_IN_EXISTS, SQL_SUBQUERIES_IN_INS, SQL_SUBQUERIES_IN_QUANTIFIEDS]; + // - ... + // Valid SQL subqueries are described under `arrow.flight.protocol.sql.SqlSupportedSubqueries`. + SqlInfo_SQL_SUPPORTED_SUBQUERIES SqlInfo = 538 + // + // Retrieves a boolean value indicating whether correlated subqueries are supported. + // + // Returns: + // - false: if correlated subqueries are unsupported; + // - true: if correlated subqueries are supported. + SqlInfo_SQL_CORRELATED_SUBQUERIES_SUPPORTED SqlInfo = 539 + // + // Retrieves the supported SQL UNIONs. + // + // Returns an int32 bitmask value representing the supported SQL UNIONs. + // The returned bitmask should be parsed in order to retrieve the supported SQL UNIONs. + // + // For instance: + // - return 0 (\b0) => [] (no supported SQL positioned commands); + // - return 1 (\b1) => [SQL_UNION]; + // - return 2 (\b10) => [SQL_UNION_ALL]; + // - return 3 (\b11) => [SQL_UNION, SQL_UNION_ALL]. + // Valid SQL positioned commands are described under `arrow.flight.protocol.sql.SqlSupportedUnions`. + SqlInfo_SQL_SUPPORTED_UNIONS SqlInfo = 540 + // Retrieves a int64 value representing the maximum number of hex characters allowed in an inline binary literal. + SqlInfo_SQL_MAX_BINARY_LITERAL_LENGTH SqlInfo = 541 + // Retrieves a int64 value representing the maximum number of characters allowed for a character literal. + SqlInfo_SQL_MAX_CHAR_LITERAL_LENGTH SqlInfo = 542 + // Retrieves a int64 value representing the maximum number of characters allowed for a column name. + SqlInfo_SQL_MAX_COLUMN_NAME_LENGTH SqlInfo = 543 + // Retrieves a int64 value representing the the maximum number of columns allowed in a GROUP BY clause. + SqlInfo_SQL_MAX_COLUMNS_IN_GROUP_BY SqlInfo = 544 + // Retrieves a int64 value representing the maximum number of columns allowed in an index. + SqlInfo_SQL_MAX_COLUMNS_IN_INDEX SqlInfo = 545 + // Retrieves a int64 value representing the maximum number of columns allowed in an ORDER BY clause. + SqlInfo_SQL_MAX_COLUMNS_IN_ORDER_BY SqlInfo = 546 + // Retrieves a int64 value representing the maximum number of columns allowed in a SELECT list. + SqlInfo_SQL_MAX_COLUMNS_IN_SELECT SqlInfo = 547 + // Retrieves a int64 value representing the maximum number of columns allowed in a table. + SqlInfo_SQL_MAX_COLUMNS_IN_TABLE SqlInfo = 548 + // Retrieves a int64 value representing the maximum number of concurrent connections possible. + SqlInfo_SQL_MAX_CONNECTIONS SqlInfo = 549 + // Retrieves a int64 value the maximum number of characters allowed in a cursor name. + SqlInfo_SQL_MAX_CURSOR_NAME_LENGTH SqlInfo = 550 + // + // Retrieves a int64 value representing the maximum number of bytes allowed for an index, + // including all of the parts of the index. + SqlInfo_SQL_MAX_INDEX_LENGTH SqlInfo = 551 + // Retrieves a int64 value representing the maximum number of characters allowed in a schema name. + SqlInfo_SQL_DB_SCHEMA_NAME_LENGTH SqlInfo = 552 + // Retrieves a int64 value representing the maximum number of characters allowed in a procedure name. + SqlInfo_SQL_MAX_PROCEDURE_NAME_LENGTH SqlInfo = 553 + // Retrieves a int64 value representing the maximum number of characters allowed in a catalog name. + SqlInfo_SQL_MAX_CATALOG_NAME_LENGTH SqlInfo = 554 + // Retrieves a int64 value representing the maximum number of bytes allowed in a single row. + SqlInfo_SQL_MAX_ROW_SIZE SqlInfo = 555 + // + // Retrieves a boolean indicating whether the return value for the JDBC method getMaxRowSize includes the SQL + // data types LONGVARCHAR and LONGVARBINARY. + // + // Returns: + // - false: if return value for the JDBC method getMaxRowSize does + // not include the SQL data types LONGVARCHAR and LONGVARBINARY; + // - true: if return value for the JDBC method getMaxRowSize includes + // the SQL data types LONGVARCHAR and LONGVARBINARY. + SqlInfo_SQL_MAX_ROW_SIZE_INCLUDES_BLOBS SqlInfo = 556 + // + // Retrieves a int64 value representing the maximum number of characters allowed for an SQL statement; + // a result of 0 (zero) means that there is no limit or the limit is not known. + SqlInfo_SQL_MAX_STATEMENT_LENGTH SqlInfo = 557 + // Retrieves a int64 value representing the maximum number of active statements that can be open at the same time. + SqlInfo_SQL_MAX_STATEMENTS SqlInfo = 558 + // Retrieves a int64 value representing the maximum number of characters allowed in a table name. + SqlInfo_SQL_MAX_TABLE_NAME_LENGTH SqlInfo = 559 + // Retrieves a int64 value representing the maximum number of tables allowed in a SELECT statement. + SqlInfo_SQL_MAX_TABLES_IN_SELECT SqlInfo = 560 + // Retrieves a int64 value representing the maximum number of characters allowed in a user name. + SqlInfo_SQL_MAX_USERNAME_LENGTH SqlInfo = 561 + // + // Retrieves this database's default transaction isolation level as described in + // `arrow.flight.protocol.sql.SqlTransactionIsolationLevel`. + // + // Returns a int32 ordinal for the SQL transaction isolation level. + SqlInfo_SQL_DEFAULT_TRANSACTION_ISOLATION SqlInfo = 562 + // + // Retrieves a boolean value indicating whether transactions are supported. If not, invoking the method commit is a + // noop, and the isolation level is `arrow.flight.protocol.sql.SqlTransactionIsolationLevel.TRANSACTION_NONE`. + // + // Returns: + // - false: if transactions are unsupported; + // - true: if transactions are supported. + SqlInfo_SQL_TRANSACTIONS_SUPPORTED SqlInfo = 563 + // + // Retrieves the supported transactions isolation levels. + // + // Returns an int32 bitmask value representing the supported transactions isolation levels. + // The returned bitmask should be parsed in order to retrieve the supported transactions isolation levels. + // + // For instance: + // - return 0 (\b0) => [] (no supported SQL transactions isolation levels); + // - return 1 (\b1) => [SQL_TRANSACTION_NONE]; + // - return 2 (\b10) => [SQL_TRANSACTION_READ_UNCOMMITTED]; + // - return 3 (\b11) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_READ_UNCOMMITTED]; + // - return 4 (\b100) => [SQL_TRANSACTION_REPEATABLE_READ]; + // - return 5 (\b101) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 6 (\b110) => [SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 7 (\b111) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 8 (\b1000) => [SQL_TRANSACTION_REPEATABLE_READ]; + // - return 9 (\b1001) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 10 (\b1010) => [SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 11 (\b1011) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 12 (\b1100) => [SQL_TRANSACTION_REPEATABLE_READ, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 13 (\b1101) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_REPEATABLE_READ, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 14 (\b1110) => [SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 15 (\b1111) => [SQL_TRANSACTION_NONE, SQL_TRANSACTION_READ_UNCOMMITTED, SQL_TRANSACTION_REPEATABLE_READ, SQL_TRANSACTION_REPEATABLE_READ]; + // - return 16 (\b10000) => [SQL_TRANSACTION_SERIALIZABLE]; + // - ... + // Valid SQL positioned commands are described under `arrow.flight.protocol.sql.SqlTransactionIsolationLevel`. + SqlInfo_SQL_SUPPORTED_TRANSACTIONS_ISOLATION_LEVELS SqlInfo = 564 + // + // Retrieves a boolean value indicating whether a data definition statement within a transaction forces + // the transaction to commit. + // + // Returns: + // - false: if a data definition statement within a transaction does not force the transaction to commit; + // - true: if a data definition statement within a transaction forces the transaction to commit. + SqlInfo_SQL_DATA_DEFINITION_CAUSES_TRANSACTION_COMMIT SqlInfo = 565 + // + // Retrieves a boolean value indicating whether a data definition statement within a transaction is ignored. + // + // Returns: + // - false: if a data definition statement within a transaction is taken into account; + // - true: a data definition statement within a transaction is ignored. + SqlInfo_SQL_DATA_DEFINITIONS_IN_TRANSACTIONS_IGNORED SqlInfo = 566 + // + // Retrieves an int32 bitmask value representing the supported result set types. + // The returned bitmask should be parsed in order to retrieve the supported result set types. + // + // For instance: + // - return 0 (\b0) => [] (no supported result set types); + // - return 1 (\b1) => [SQL_RESULT_SET_TYPE_UNSPECIFIED]; + // - return 2 (\b10) => [SQL_RESULT_SET_TYPE_FORWARD_ONLY]; + // - return 3 (\b11) => [SQL_RESULT_SET_TYPE_UNSPECIFIED, SQL_RESULT_SET_TYPE_FORWARD_ONLY]; + // - return 4 (\b100) => [SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE]; + // - return 5 (\b101) => [SQL_RESULT_SET_TYPE_UNSPECIFIED, SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE]; + // - return 6 (\b110) => [SQL_RESULT_SET_TYPE_FORWARD_ONLY, SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE]; + // - return 7 (\b111) => [SQL_RESULT_SET_TYPE_UNSPECIFIED, SQL_RESULT_SET_TYPE_FORWARD_ONLY, SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE]; + // - return 8 (\b1000) => [SQL_RESULT_SET_TYPE_SCROLL_SENSITIVE]; + // - ... + // Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetType`. + SqlInfo_SQL_SUPPORTED_RESULT_SET_TYPES SqlInfo = 567 + // + // Returns an int32 bitmask value concurrency types supported for + // `arrow.flight.protocol.sql.SqlSupportedResultSetType.SQL_RESULT_SET_TYPE_UNSPECIFIED`. + // + // For instance: + // - return 0 (\b0) => [] (no supported concurrency types for this result set type) + // - return 1 (\b1) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED] + // - return 2 (\b10) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 3 (\b11) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 4 (\b100) => [SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 5 (\b101) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 6 (\b110) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 7 (\b111) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency`. + SqlInfo_SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_UNSPECIFIED SqlInfo = 568 + // + // Returns an int32 bitmask value concurrency types supported for + // `arrow.flight.protocol.sql.SqlSupportedResultSetType.SQL_RESULT_SET_TYPE_FORWARD_ONLY`. + // + // For instance: + // - return 0 (\b0) => [] (no supported concurrency types for this result set type) + // - return 1 (\b1) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED] + // - return 2 (\b10) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 3 (\b11) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 4 (\b100) => [SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 5 (\b101) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 6 (\b110) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 7 (\b111) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency`. + SqlInfo_SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_FORWARD_ONLY SqlInfo = 569 + // + // Returns an int32 bitmask value concurrency types supported for + // `arrow.flight.protocol.sql.SqlSupportedResultSetType.SQL_RESULT_SET_TYPE_SCROLL_SENSITIVE`. + // + // For instance: + // - return 0 (\b0) => [] (no supported concurrency types for this result set type) + // - return 1 (\b1) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED] + // - return 2 (\b10) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 3 (\b11) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 4 (\b100) => [SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 5 (\b101) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 6 (\b110) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 7 (\b111) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency`. + SqlInfo_SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_SCROLL_SENSITIVE SqlInfo = 570 + // + // Returns an int32 bitmask value concurrency types supported for + // `arrow.flight.protocol.sql.SqlSupportedResultSetType.SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE`. + // + // For instance: + // - return 0 (\b0) => [] (no supported concurrency types for this result set type) + // - return 1 (\b1) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED] + // - return 2 (\b10) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 3 (\b11) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY] + // - return 4 (\b100) => [SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 5 (\b101) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 6 (\b110) => [SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // - return 7 (\b111) => [SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED, SQL_RESULT_SET_CONCURRENCY_READ_ONLY, SQL_RESULT_SET_CONCURRENCY_UPDATABLE] + // Valid result set types are described under `arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency`. + SqlInfo_SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_SCROLL_INSENSITIVE SqlInfo = 571 + // + // Retrieves a boolean value indicating whether this database supports batch updates. + // + // - false: if this database does not support batch updates; + // - true: if this database supports batch updates. + SqlInfo_SQL_BATCH_UPDATES_SUPPORTED SqlInfo = 572 + // + // Retrieves a boolean value indicating whether this database supports savepoints. + // + // Returns: + // - false: if this database does not support savepoints; + // - true: if this database supports savepoints. + SqlInfo_SQL_SAVEPOINTS_SUPPORTED SqlInfo = 573 + // + // Retrieves a boolean value indicating whether named parameters are supported in callable statements. + // + // Returns: + // - false: if named parameters in callable statements are unsupported; + // - true: if named parameters in callable statements are supported. + SqlInfo_SQL_NAMED_PARAMETERS_SUPPORTED SqlInfo = 574 + // + // Retrieves a boolean value indicating whether updates made to a LOB are made on a copy or directly to the LOB. + // + // Returns: + // - false: if updates made to a LOB are made directly to the LOB; + // - true: if updates made to a LOB are made on a copy. + SqlInfo_SQL_LOCATORS_UPDATE_COPY SqlInfo = 575 + // + // Retrieves a boolean value indicating whether invoking user-defined or vendor functions + // using the stored procedure escape syntax is supported. + // + // Returns: + // - false: if invoking user-defined or vendor functions using the stored procedure escape syntax is unsupported; + // - true: if invoking user-defined or vendor functions using the stored procedure escape syntax is supported. + SqlInfo_SQL_STORED_FUNCTIONS_USING_CALL_SYNTAX_SUPPORTED SqlInfo = 576 +) + +// Enum value maps for SqlInfo. +var ( + SqlInfo_name = map[int32]string{ + 0: "FLIGHT_SQL_SERVER_NAME", + 1: "FLIGHT_SQL_SERVER_VERSION", + 2: "FLIGHT_SQL_SERVER_ARROW_VERSION", + 3: "FLIGHT_SQL_SERVER_READ_ONLY", + 500: "SQL_DDL_CATALOG", + 501: "SQL_DDL_SCHEMA", + 502: "SQL_DDL_TABLE", + 503: "SQL_IDENTIFIER_CASE", + 504: "SQL_IDENTIFIER_QUOTE_CHAR", + 505: "SQL_QUOTED_IDENTIFIER_CASE", + 506: "SQL_ALL_TABLES_ARE_SELECTABLE", + 507: "SQL_NULL_ORDERING", + 508: "SQL_KEYWORDS", + 509: "SQL_NUMERIC_FUNCTIONS", + 510: "SQL_STRING_FUNCTIONS", + 511: "SQL_SYSTEM_FUNCTIONS", + 512: "SQL_DATETIME_FUNCTIONS", + 513: "SQL_SEARCH_STRING_ESCAPE", + 514: "SQL_EXTRA_NAME_CHARACTERS", + 515: "SQL_SUPPORTS_COLUMN_ALIASING", + 516: "SQL_NULL_PLUS_NULL_IS_NULL", + 517: "SQL_SUPPORTS_CONVERT", + 518: "SQL_SUPPORTS_TABLE_CORRELATION_NAMES", + 519: "SQL_SUPPORTS_DIFFERENT_TABLE_CORRELATION_NAMES", + 520: "SQL_SUPPORTS_EXPRESSIONS_IN_ORDER_BY", + 521: "SQL_SUPPORTS_ORDER_BY_UNRELATED", + 522: "SQL_SUPPORTED_GROUP_BY", + 523: "SQL_SUPPORTS_LIKE_ESCAPE_CLAUSE", + 524: "SQL_SUPPORTS_NON_NULLABLE_COLUMNS", + 525: "SQL_SUPPORTED_GRAMMAR", + 526: "SQL_ANSI92_SUPPORTED_LEVEL", + 527: "SQL_SUPPORTS_INTEGRITY_ENHANCEMENT_FACILITY", + 528: "SQL_OUTER_JOINS_SUPPORT_LEVEL", + 529: "SQL_SCHEMA_TERM", + 530: "SQL_PROCEDURE_TERM", + 531: "SQL_CATALOG_TERM", + 532: "SQL_CATALOG_AT_START", + 533: "SQL_SCHEMAS_SUPPORTED_ACTIONS", + 534: "SQL_CATALOGS_SUPPORTED_ACTIONS", + 535: "SQL_SUPPORTED_POSITIONED_COMMANDS", + 536: "SQL_SELECT_FOR_UPDATE_SUPPORTED", + 537: "SQL_STORED_PROCEDURES_SUPPORTED", + 538: "SQL_SUPPORTED_SUBQUERIES", + 539: "SQL_CORRELATED_SUBQUERIES_SUPPORTED", + 540: "SQL_SUPPORTED_UNIONS", + 541: "SQL_MAX_BINARY_LITERAL_LENGTH", + 542: "SQL_MAX_CHAR_LITERAL_LENGTH", + 543: "SQL_MAX_COLUMN_NAME_LENGTH", + 544: "SQL_MAX_COLUMNS_IN_GROUP_BY", + 545: "SQL_MAX_COLUMNS_IN_INDEX", + 546: "SQL_MAX_COLUMNS_IN_ORDER_BY", + 547: "SQL_MAX_COLUMNS_IN_SELECT", + 548: "SQL_MAX_COLUMNS_IN_TABLE", + 549: "SQL_MAX_CONNECTIONS", + 550: "SQL_MAX_CURSOR_NAME_LENGTH", + 551: "SQL_MAX_INDEX_LENGTH", + 552: "SQL_DB_SCHEMA_NAME_LENGTH", + 553: "SQL_MAX_PROCEDURE_NAME_LENGTH", + 554: "SQL_MAX_CATALOG_NAME_LENGTH", + 555: "SQL_MAX_ROW_SIZE", + 556: "SQL_MAX_ROW_SIZE_INCLUDES_BLOBS", + 557: "SQL_MAX_STATEMENT_LENGTH", + 558: "SQL_MAX_STATEMENTS", + 559: "SQL_MAX_TABLE_NAME_LENGTH", + 560: "SQL_MAX_TABLES_IN_SELECT", + 561: "SQL_MAX_USERNAME_LENGTH", + 562: "SQL_DEFAULT_TRANSACTION_ISOLATION", + 563: "SQL_TRANSACTIONS_SUPPORTED", + 564: "SQL_SUPPORTED_TRANSACTIONS_ISOLATION_LEVELS", + 565: "SQL_DATA_DEFINITION_CAUSES_TRANSACTION_COMMIT", + 566: "SQL_DATA_DEFINITIONS_IN_TRANSACTIONS_IGNORED", + 567: "SQL_SUPPORTED_RESULT_SET_TYPES", + 568: "SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_UNSPECIFIED", + 569: "SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_FORWARD_ONLY", + 570: "SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_SCROLL_SENSITIVE", + 571: "SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_SCROLL_INSENSITIVE", + 572: "SQL_BATCH_UPDATES_SUPPORTED", + 573: "SQL_SAVEPOINTS_SUPPORTED", + 574: "SQL_NAMED_PARAMETERS_SUPPORTED", + 575: "SQL_LOCATORS_UPDATE_COPY", + 576: "SQL_STORED_FUNCTIONS_USING_CALL_SYNTAX_SUPPORTED", + } + SqlInfo_value = map[string]int32{ + "FLIGHT_SQL_SERVER_NAME": 0, + "FLIGHT_SQL_SERVER_VERSION": 1, + "FLIGHT_SQL_SERVER_ARROW_VERSION": 2, + "FLIGHT_SQL_SERVER_READ_ONLY": 3, + "SQL_DDL_CATALOG": 500, + "SQL_DDL_SCHEMA": 501, + "SQL_DDL_TABLE": 502, + "SQL_IDENTIFIER_CASE": 503, + "SQL_IDENTIFIER_QUOTE_CHAR": 504, + "SQL_QUOTED_IDENTIFIER_CASE": 505, + "SQL_ALL_TABLES_ARE_SELECTABLE": 506, + "SQL_NULL_ORDERING": 507, + "SQL_KEYWORDS": 508, + "SQL_NUMERIC_FUNCTIONS": 509, + "SQL_STRING_FUNCTIONS": 510, + "SQL_SYSTEM_FUNCTIONS": 511, + "SQL_DATETIME_FUNCTIONS": 512, + "SQL_SEARCH_STRING_ESCAPE": 513, + "SQL_EXTRA_NAME_CHARACTERS": 514, + "SQL_SUPPORTS_COLUMN_ALIASING": 515, + "SQL_NULL_PLUS_NULL_IS_NULL": 516, + "SQL_SUPPORTS_CONVERT": 517, + "SQL_SUPPORTS_TABLE_CORRELATION_NAMES": 518, + "SQL_SUPPORTS_DIFFERENT_TABLE_CORRELATION_NAMES": 519, + "SQL_SUPPORTS_EXPRESSIONS_IN_ORDER_BY": 520, + "SQL_SUPPORTS_ORDER_BY_UNRELATED": 521, + "SQL_SUPPORTED_GROUP_BY": 522, + "SQL_SUPPORTS_LIKE_ESCAPE_CLAUSE": 523, + "SQL_SUPPORTS_NON_NULLABLE_COLUMNS": 524, + "SQL_SUPPORTED_GRAMMAR": 525, + "SQL_ANSI92_SUPPORTED_LEVEL": 526, + "SQL_SUPPORTS_INTEGRITY_ENHANCEMENT_FACILITY": 527, + "SQL_OUTER_JOINS_SUPPORT_LEVEL": 528, + "SQL_SCHEMA_TERM": 529, + "SQL_PROCEDURE_TERM": 530, + "SQL_CATALOG_TERM": 531, + "SQL_CATALOG_AT_START": 532, + "SQL_SCHEMAS_SUPPORTED_ACTIONS": 533, + "SQL_CATALOGS_SUPPORTED_ACTIONS": 534, + "SQL_SUPPORTED_POSITIONED_COMMANDS": 535, + "SQL_SELECT_FOR_UPDATE_SUPPORTED": 536, + "SQL_STORED_PROCEDURES_SUPPORTED": 537, + "SQL_SUPPORTED_SUBQUERIES": 538, + "SQL_CORRELATED_SUBQUERIES_SUPPORTED": 539, + "SQL_SUPPORTED_UNIONS": 540, + "SQL_MAX_BINARY_LITERAL_LENGTH": 541, + "SQL_MAX_CHAR_LITERAL_LENGTH": 542, + "SQL_MAX_COLUMN_NAME_LENGTH": 543, + "SQL_MAX_COLUMNS_IN_GROUP_BY": 544, + "SQL_MAX_COLUMNS_IN_INDEX": 545, + "SQL_MAX_COLUMNS_IN_ORDER_BY": 546, + "SQL_MAX_COLUMNS_IN_SELECT": 547, + "SQL_MAX_COLUMNS_IN_TABLE": 548, + "SQL_MAX_CONNECTIONS": 549, + "SQL_MAX_CURSOR_NAME_LENGTH": 550, + "SQL_MAX_INDEX_LENGTH": 551, + "SQL_DB_SCHEMA_NAME_LENGTH": 552, + "SQL_MAX_PROCEDURE_NAME_LENGTH": 553, + "SQL_MAX_CATALOG_NAME_LENGTH": 554, + "SQL_MAX_ROW_SIZE": 555, + "SQL_MAX_ROW_SIZE_INCLUDES_BLOBS": 556, + "SQL_MAX_STATEMENT_LENGTH": 557, + "SQL_MAX_STATEMENTS": 558, + "SQL_MAX_TABLE_NAME_LENGTH": 559, + "SQL_MAX_TABLES_IN_SELECT": 560, + "SQL_MAX_USERNAME_LENGTH": 561, + "SQL_DEFAULT_TRANSACTION_ISOLATION": 562, + "SQL_TRANSACTIONS_SUPPORTED": 563, + "SQL_SUPPORTED_TRANSACTIONS_ISOLATION_LEVELS": 564, + "SQL_DATA_DEFINITION_CAUSES_TRANSACTION_COMMIT": 565, + "SQL_DATA_DEFINITIONS_IN_TRANSACTIONS_IGNORED": 566, + "SQL_SUPPORTED_RESULT_SET_TYPES": 567, + "SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_UNSPECIFIED": 568, + "SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_FORWARD_ONLY": 569, + "SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_SCROLL_SENSITIVE": 570, + "SQL_SUPPORTED_CONCURRENCIES_FOR_RESULT_SET_SCROLL_INSENSITIVE": 571, + "SQL_BATCH_UPDATES_SUPPORTED": 572, + "SQL_SAVEPOINTS_SUPPORTED": 573, + "SQL_NAMED_PARAMETERS_SUPPORTED": 574, + "SQL_LOCATORS_UPDATE_COPY": 575, + "SQL_STORED_FUNCTIONS_USING_CALL_SYNTAX_SUPPORTED": 576, + } +) + +func (x SqlInfo) Enum() *SqlInfo { + p := new(SqlInfo) + *p = x + return p +} + +func (x SqlInfo) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlInfo) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[0].Descriptor() +} + +func (SqlInfo) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[0] +} + +func (x SqlInfo) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlInfo.Descriptor instead. +func (SqlInfo) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{0} +} + +type SqlSupportedCaseSensitivity int32 + +const ( + SqlSupportedCaseSensitivity_SQL_CASE_SENSITIVITY_UNKNOWN SqlSupportedCaseSensitivity = 0 + SqlSupportedCaseSensitivity_SQL_CASE_SENSITIVITY_CASE_INSENSITIVE SqlSupportedCaseSensitivity = 1 + SqlSupportedCaseSensitivity_SQL_CASE_SENSITIVITY_UPPERCASE SqlSupportedCaseSensitivity = 2 + SqlSupportedCaseSensitivity_SQL_CASE_SENSITIVITY_LOWERCASE SqlSupportedCaseSensitivity = 3 +) + +// Enum value maps for SqlSupportedCaseSensitivity. +var ( + SqlSupportedCaseSensitivity_name = map[int32]string{ + 0: "SQL_CASE_SENSITIVITY_UNKNOWN", + 1: "SQL_CASE_SENSITIVITY_CASE_INSENSITIVE", + 2: "SQL_CASE_SENSITIVITY_UPPERCASE", + 3: "SQL_CASE_SENSITIVITY_LOWERCASE", + } + SqlSupportedCaseSensitivity_value = map[string]int32{ + "SQL_CASE_SENSITIVITY_UNKNOWN": 0, + "SQL_CASE_SENSITIVITY_CASE_INSENSITIVE": 1, + "SQL_CASE_SENSITIVITY_UPPERCASE": 2, + "SQL_CASE_SENSITIVITY_LOWERCASE": 3, + } +) + +func (x SqlSupportedCaseSensitivity) Enum() *SqlSupportedCaseSensitivity { + p := new(SqlSupportedCaseSensitivity) + *p = x + return p +} + +func (x SqlSupportedCaseSensitivity) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlSupportedCaseSensitivity) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[1].Descriptor() +} + +func (SqlSupportedCaseSensitivity) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[1] +} + +func (x SqlSupportedCaseSensitivity) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlSupportedCaseSensitivity.Descriptor instead. +func (SqlSupportedCaseSensitivity) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{1} +} + +type SqlNullOrdering int32 + +const ( + SqlNullOrdering_SQL_NULLS_SORTED_HIGH SqlNullOrdering = 0 + SqlNullOrdering_SQL_NULLS_SORTED_LOW SqlNullOrdering = 1 + SqlNullOrdering_SQL_NULLS_SORTED_AT_START SqlNullOrdering = 2 + SqlNullOrdering_SQL_NULLS_SORTED_AT_END SqlNullOrdering = 3 +) + +// Enum value maps for SqlNullOrdering. +var ( + SqlNullOrdering_name = map[int32]string{ + 0: "SQL_NULLS_SORTED_HIGH", + 1: "SQL_NULLS_SORTED_LOW", + 2: "SQL_NULLS_SORTED_AT_START", + 3: "SQL_NULLS_SORTED_AT_END", + } + SqlNullOrdering_value = map[string]int32{ + "SQL_NULLS_SORTED_HIGH": 0, + "SQL_NULLS_SORTED_LOW": 1, + "SQL_NULLS_SORTED_AT_START": 2, + "SQL_NULLS_SORTED_AT_END": 3, + } +) + +func (x SqlNullOrdering) Enum() *SqlNullOrdering { + p := new(SqlNullOrdering) + *p = x + return p +} + +func (x SqlNullOrdering) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlNullOrdering) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[2].Descriptor() +} + +func (SqlNullOrdering) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[2] +} + +func (x SqlNullOrdering) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlNullOrdering.Descriptor instead. +func (SqlNullOrdering) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{2} +} + +type SupportedSqlGrammar int32 + +const ( + SupportedSqlGrammar_SQL_MINIMUM_GRAMMAR SupportedSqlGrammar = 0 + SupportedSqlGrammar_SQL_CORE_GRAMMAR SupportedSqlGrammar = 1 + SupportedSqlGrammar_SQL_EXTENDED_GRAMMAR SupportedSqlGrammar = 2 +) + +// Enum value maps for SupportedSqlGrammar. +var ( + SupportedSqlGrammar_name = map[int32]string{ + 0: "SQL_MINIMUM_GRAMMAR", + 1: "SQL_CORE_GRAMMAR", + 2: "SQL_EXTENDED_GRAMMAR", + } + SupportedSqlGrammar_value = map[string]int32{ + "SQL_MINIMUM_GRAMMAR": 0, + "SQL_CORE_GRAMMAR": 1, + "SQL_EXTENDED_GRAMMAR": 2, + } +) + +func (x SupportedSqlGrammar) Enum() *SupportedSqlGrammar { + p := new(SupportedSqlGrammar) + *p = x + return p +} + +func (x SupportedSqlGrammar) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SupportedSqlGrammar) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[3].Descriptor() +} + +func (SupportedSqlGrammar) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[3] +} + +func (x SupportedSqlGrammar) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SupportedSqlGrammar.Descriptor instead. +func (SupportedSqlGrammar) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{3} +} + +type SupportedAnsi92SqlGrammarLevel int32 + +const ( + SupportedAnsi92SqlGrammarLevel_ANSI92_ENTRY_SQL SupportedAnsi92SqlGrammarLevel = 0 + SupportedAnsi92SqlGrammarLevel_ANSI92_INTERMEDIATE_SQL SupportedAnsi92SqlGrammarLevel = 1 + SupportedAnsi92SqlGrammarLevel_ANSI92_FULL_SQL SupportedAnsi92SqlGrammarLevel = 2 +) + +// Enum value maps for SupportedAnsi92SqlGrammarLevel. +var ( + SupportedAnsi92SqlGrammarLevel_name = map[int32]string{ + 0: "ANSI92_ENTRY_SQL", + 1: "ANSI92_INTERMEDIATE_SQL", + 2: "ANSI92_FULL_SQL", + } + SupportedAnsi92SqlGrammarLevel_value = map[string]int32{ + "ANSI92_ENTRY_SQL": 0, + "ANSI92_INTERMEDIATE_SQL": 1, + "ANSI92_FULL_SQL": 2, + } +) + +func (x SupportedAnsi92SqlGrammarLevel) Enum() *SupportedAnsi92SqlGrammarLevel { + p := new(SupportedAnsi92SqlGrammarLevel) + *p = x + return p +} + +func (x SupportedAnsi92SqlGrammarLevel) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SupportedAnsi92SqlGrammarLevel) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[4].Descriptor() +} + +func (SupportedAnsi92SqlGrammarLevel) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[4] +} + +func (x SupportedAnsi92SqlGrammarLevel) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SupportedAnsi92SqlGrammarLevel.Descriptor instead. +func (SupportedAnsi92SqlGrammarLevel) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{4} +} + +type SqlOuterJoinsSupportLevel int32 + +const ( + SqlOuterJoinsSupportLevel_SQL_JOINS_UNSUPPORTED SqlOuterJoinsSupportLevel = 0 + SqlOuterJoinsSupportLevel_SQL_LIMITED_OUTER_JOINS SqlOuterJoinsSupportLevel = 1 + SqlOuterJoinsSupportLevel_SQL_FULL_OUTER_JOINS SqlOuterJoinsSupportLevel = 2 +) + +// Enum value maps for SqlOuterJoinsSupportLevel. +var ( + SqlOuterJoinsSupportLevel_name = map[int32]string{ + 0: "SQL_JOINS_UNSUPPORTED", + 1: "SQL_LIMITED_OUTER_JOINS", + 2: "SQL_FULL_OUTER_JOINS", + } + SqlOuterJoinsSupportLevel_value = map[string]int32{ + "SQL_JOINS_UNSUPPORTED": 0, + "SQL_LIMITED_OUTER_JOINS": 1, + "SQL_FULL_OUTER_JOINS": 2, + } +) + +func (x SqlOuterJoinsSupportLevel) Enum() *SqlOuterJoinsSupportLevel { + p := new(SqlOuterJoinsSupportLevel) + *p = x + return p +} + +func (x SqlOuterJoinsSupportLevel) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlOuterJoinsSupportLevel) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[5].Descriptor() +} + +func (SqlOuterJoinsSupportLevel) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[5] +} + +func (x SqlOuterJoinsSupportLevel) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlOuterJoinsSupportLevel.Descriptor instead. +func (SqlOuterJoinsSupportLevel) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{5} +} + +type SqlSupportedGroupBy int32 + +const ( + SqlSupportedGroupBy_SQL_GROUP_BY_UNRELATED SqlSupportedGroupBy = 0 + SqlSupportedGroupBy_SQL_GROUP_BY_BEYOND_SELECT SqlSupportedGroupBy = 1 +) + +// Enum value maps for SqlSupportedGroupBy. +var ( + SqlSupportedGroupBy_name = map[int32]string{ + 0: "SQL_GROUP_BY_UNRELATED", + 1: "SQL_GROUP_BY_BEYOND_SELECT", + } + SqlSupportedGroupBy_value = map[string]int32{ + "SQL_GROUP_BY_UNRELATED": 0, + "SQL_GROUP_BY_BEYOND_SELECT": 1, + } +) + +func (x SqlSupportedGroupBy) Enum() *SqlSupportedGroupBy { + p := new(SqlSupportedGroupBy) + *p = x + return p +} + +func (x SqlSupportedGroupBy) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlSupportedGroupBy) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[6].Descriptor() +} + +func (SqlSupportedGroupBy) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[6] +} + +func (x SqlSupportedGroupBy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlSupportedGroupBy.Descriptor instead. +func (SqlSupportedGroupBy) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{6} +} + +type SqlSupportedElementActions int32 + +const ( + SqlSupportedElementActions_SQL_ELEMENT_IN_PROCEDURE_CALLS SqlSupportedElementActions = 0 + SqlSupportedElementActions_SQL_ELEMENT_IN_INDEX_DEFINITIONS SqlSupportedElementActions = 1 + SqlSupportedElementActions_SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS SqlSupportedElementActions = 2 +) + +// Enum value maps for SqlSupportedElementActions. +var ( + SqlSupportedElementActions_name = map[int32]string{ + 0: "SQL_ELEMENT_IN_PROCEDURE_CALLS", + 1: "SQL_ELEMENT_IN_INDEX_DEFINITIONS", + 2: "SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS", + } + SqlSupportedElementActions_value = map[string]int32{ + "SQL_ELEMENT_IN_PROCEDURE_CALLS": 0, + "SQL_ELEMENT_IN_INDEX_DEFINITIONS": 1, + "SQL_ELEMENT_IN_PRIVILEGE_DEFINITIONS": 2, + } +) + +func (x SqlSupportedElementActions) Enum() *SqlSupportedElementActions { + p := new(SqlSupportedElementActions) + *p = x + return p +} + +func (x SqlSupportedElementActions) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlSupportedElementActions) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[7].Descriptor() +} + +func (SqlSupportedElementActions) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[7] +} + +func (x SqlSupportedElementActions) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlSupportedElementActions.Descriptor instead. +func (SqlSupportedElementActions) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{7} +} + +type SqlSupportedPositionedCommands int32 + +const ( + SqlSupportedPositionedCommands_SQL_POSITIONED_DELETE SqlSupportedPositionedCommands = 0 + SqlSupportedPositionedCommands_SQL_POSITIONED_UPDATE SqlSupportedPositionedCommands = 1 +) + +// Enum value maps for SqlSupportedPositionedCommands. +var ( + SqlSupportedPositionedCommands_name = map[int32]string{ + 0: "SQL_POSITIONED_DELETE", + 1: "SQL_POSITIONED_UPDATE", + } + SqlSupportedPositionedCommands_value = map[string]int32{ + "SQL_POSITIONED_DELETE": 0, + "SQL_POSITIONED_UPDATE": 1, + } +) + +func (x SqlSupportedPositionedCommands) Enum() *SqlSupportedPositionedCommands { + p := new(SqlSupportedPositionedCommands) + *p = x + return p +} + +func (x SqlSupportedPositionedCommands) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlSupportedPositionedCommands) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[8].Descriptor() +} + +func (SqlSupportedPositionedCommands) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[8] +} + +func (x SqlSupportedPositionedCommands) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlSupportedPositionedCommands.Descriptor instead. +func (SqlSupportedPositionedCommands) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{8} +} + +type SqlSupportedSubqueries int32 + +const ( + SqlSupportedSubqueries_SQL_SUBQUERIES_IN_COMPARISONS SqlSupportedSubqueries = 0 + SqlSupportedSubqueries_SQL_SUBQUERIES_IN_EXISTS SqlSupportedSubqueries = 1 + SqlSupportedSubqueries_SQL_SUBQUERIES_IN_INS SqlSupportedSubqueries = 2 + SqlSupportedSubqueries_SQL_SUBQUERIES_IN_QUANTIFIEDS SqlSupportedSubqueries = 3 +) + +// Enum value maps for SqlSupportedSubqueries. +var ( + SqlSupportedSubqueries_name = map[int32]string{ + 0: "SQL_SUBQUERIES_IN_COMPARISONS", + 1: "SQL_SUBQUERIES_IN_EXISTS", + 2: "SQL_SUBQUERIES_IN_INS", + 3: "SQL_SUBQUERIES_IN_QUANTIFIEDS", + } + SqlSupportedSubqueries_value = map[string]int32{ + "SQL_SUBQUERIES_IN_COMPARISONS": 0, + "SQL_SUBQUERIES_IN_EXISTS": 1, + "SQL_SUBQUERIES_IN_INS": 2, + "SQL_SUBQUERIES_IN_QUANTIFIEDS": 3, + } +) + +func (x SqlSupportedSubqueries) Enum() *SqlSupportedSubqueries { + p := new(SqlSupportedSubqueries) + *p = x + return p +} + +func (x SqlSupportedSubqueries) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlSupportedSubqueries) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[9].Descriptor() +} + +func (SqlSupportedSubqueries) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[9] +} + +func (x SqlSupportedSubqueries) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlSupportedSubqueries.Descriptor instead. +func (SqlSupportedSubqueries) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{9} +} + +type SqlSupportedUnions int32 + +const ( + SqlSupportedUnions_SQL_UNION SqlSupportedUnions = 0 + SqlSupportedUnions_SQL_UNION_ALL SqlSupportedUnions = 1 +) + +// Enum value maps for SqlSupportedUnions. +var ( + SqlSupportedUnions_name = map[int32]string{ + 0: "SQL_UNION", + 1: "SQL_UNION_ALL", + } + SqlSupportedUnions_value = map[string]int32{ + "SQL_UNION": 0, + "SQL_UNION_ALL": 1, + } +) + +func (x SqlSupportedUnions) Enum() *SqlSupportedUnions { + p := new(SqlSupportedUnions) + *p = x + return p +} + +func (x SqlSupportedUnions) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlSupportedUnions) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[10].Descriptor() +} + +func (SqlSupportedUnions) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[10] +} + +func (x SqlSupportedUnions) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlSupportedUnions.Descriptor instead. +func (SqlSupportedUnions) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{10} +} + +type SqlTransactionIsolationLevel int32 + +const ( + SqlTransactionIsolationLevel_SQL_TRANSACTION_NONE SqlTransactionIsolationLevel = 0 + SqlTransactionIsolationLevel_SQL_TRANSACTION_READ_UNCOMMITTED SqlTransactionIsolationLevel = 1 + SqlTransactionIsolationLevel_SQL_TRANSACTION_READ_COMMITTED SqlTransactionIsolationLevel = 2 + SqlTransactionIsolationLevel_SQL_TRANSACTION_REPEATABLE_READ SqlTransactionIsolationLevel = 3 + SqlTransactionIsolationLevel_SQL_TRANSACTION_SERIALIZABLE SqlTransactionIsolationLevel = 4 +) + +// Enum value maps for SqlTransactionIsolationLevel. +var ( + SqlTransactionIsolationLevel_name = map[int32]string{ + 0: "SQL_TRANSACTION_NONE", + 1: "SQL_TRANSACTION_READ_UNCOMMITTED", + 2: "SQL_TRANSACTION_READ_COMMITTED", + 3: "SQL_TRANSACTION_REPEATABLE_READ", + 4: "SQL_TRANSACTION_SERIALIZABLE", + } + SqlTransactionIsolationLevel_value = map[string]int32{ + "SQL_TRANSACTION_NONE": 0, + "SQL_TRANSACTION_READ_UNCOMMITTED": 1, + "SQL_TRANSACTION_READ_COMMITTED": 2, + "SQL_TRANSACTION_REPEATABLE_READ": 3, + "SQL_TRANSACTION_SERIALIZABLE": 4, + } +) + +func (x SqlTransactionIsolationLevel) Enum() *SqlTransactionIsolationLevel { + p := new(SqlTransactionIsolationLevel) + *p = x + return p +} + +func (x SqlTransactionIsolationLevel) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlTransactionIsolationLevel) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[11].Descriptor() +} + +func (SqlTransactionIsolationLevel) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[11] +} + +func (x SqlTransactionIsolationLevel) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlTransactionIsolationLevel.Descriptor instead. +func (SqlTransactionIsolationLevel) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{11} +} + +type SqlSupportedTransactions int32 + +const ( + SqlSupportedTransactions_SQL_TRANSACTION_UNSPECIFIED SqlSupportedTransactions = 0 + SqlSupportedTransactions_SQL_DATA_DEFINITION_TRANSACTIONS SqlSupportedTransactions = 1 + SqlSupportedTransactions_SQL_DATA_MANIPULATION_TRANSACTIONS SqlSupportedTransactions = 2 +) + +// Enum value maps for SqlSupportedTransactions. +var ( + SqlSupportedTransactions_name = map[int32]string{ + 0: "SQL_TRANSACTION_UNSPECIFIED", + 1: "SQL_DATA_DEFINITION_TRANSACTIONS", + 2: "SQL_DATA_MANIPULATION_TRANSACTIONS", + } + SqlSupportedTransactions_value = map[string]int32{ + "SQL_TRANSACTION_UNSPECIFIED": 0, + "SQL_DATA_DEFINITION_TRANSACTIONS": 1, + "SQL_DATA_MANIPULATION_TRANSACTIONS": 2, + } +) + +func (x SqlSupportedTransactions) Enum() *SqlSupportedTransactions { + p := new(SqlSupportedTransactions) + *p = x + return p +} + +func (x SqlSupportedTransactions) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlSupportedTransactions) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[12].Descriptor() +} + +func (SqlSupportedTransactions) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[12] +} + +func (x SqlSupportedTransactions) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlSupportedTransactions.Descriptor instead. +func (SqlSupportedTransactions) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{12} +} + +type SqlSupportedResultSetType int32 + +const ( + SqlSupportedResultSetType_SQL_RESULT_SET_TYPE_UNSPECIFIED SqlSupportedResultSetType = 0 + SqlSupportedResultSetType_SQL_RESULT_SET_TYPE_FORWARD_ONLY SqlSupportedResultSetType = 1 + SqlSupportedResultSetType_SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE SqlSupportedResultSetType = 2 + SqlSupportedResultSetType_SQL_RESULT_SET_TYPE_SCROLL_SENSITIVE SqlSupportedResultSetType = 3 +) + +// Enum value maps for SqlSupportedResultSetType. +var ( + SqlSupportedResultSetType_name = map[int32]string{ + 0: "SQL_RESULT_SET_TYPE_UNSPECIFIED", + 1: "SQL_RESULT_SET_TYPE_FORWARD_ONLY", + 2: "SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE", + 3: "SQL_RESULT_SET_TYPE_SCROLL_SENSITIVE", + } + SqlSupportedResultSetType_value = map[string]int32{ + "SQL_RESULT_SET_TYPE_UNSPECIFIED": 0, + "SQL_RESULT_SET_TYPE_FORWARD_ONLY": 1, + "SQL_RESULT_SET_TYPE_SCROLL_INSENSITIVE": 2, + "SQL_RESULT_SET_TYPE_SCROLL_SENSITIVE": 3, + } +) + +func (x SqlSupportedResultSetType) Enum() *SqlSupportedResultSetType { + p := new(SqlSupportedResultSetType) + *p = x + return p +} + +func (x SqlSupportedResultSetType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlSupportedResultSetType) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[13].Descriptor() +} + +func (SqlSupportedResultSetType) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[13] +} + +func (x SqlSupportedResultSetType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlSupportedResultSetType.Descriptor instead. +func (SqlSupportedResultSetType) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{13} +} + +type SqlSupportedResultSetConcurrency int32 + +const ( + SqlSupportedResultSetConcurrency_SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED SqlSupportedResultSetConcurrency = 0 + SqlSupportedResultSetConcurrency_SQL_RESULT_SET_CONCURRENCY_READ_ONLY SqlSupportedResultSetConcurrency = 1 + SqlSupportedResultSetConcurrency_SQL_RESULT_SET_CONCURRENCY_UPDATABLE SqlSupportedResultSetConcurrency = 2 +) + +// Enum value maps for SqlSupportedResultSetConcurrency. +var ( + SqlSupportedResultSetConcurrency_name = map[int32]string{ + 0: "SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED", + 1: "SQL_RESULT_SET_CONCURRENCY_READ_ONLY", + 2: "SQL_RESULT_SET_CONCURRENCY_UPDATABLE", + } + SqlSupportedResultSetConcurrency_value = map[string]int32{ + "SQL_RESULT_SET_CONCURRENCY_UNSPECIFIED": 0, + "SQL_RESULT_SET_CONCURRENCY_READ_ONLY": 1, + "SQL_RESULT_SET_CONCURRENCY_UPDATABLE": 2, + } +) + +func (x SqlSupportedResultSetConcurrency) Enum() *SqlSupportedResultSetConcurrency { + p := new(SqlSupportedResultSetConcurrency) + *p = x + return p +} + +func (x SqlSupportedResultSetConcurrency) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlSupportedResultSetConcurrency) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[14].Descriptor() +} + +func (SqlSupportedResultSetConcurrency) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[14] +} + +func (x SqlSupportedResultSetConcurrency) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlSupportedResultSetConcurrency.Descriptor instead. +func (SqlSupportedResultSetConcurrency) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{14} +} + +type SqlSupportsConvert int32 + +const ( + SqlSupportsConvert_SQL_CONVERT_BIGINT SqlSupportsConvert = 0 + SqlSupportsConvert_SQL_CONVERT_BINARY SqlSupportsConvert = 1 + SqlSupportsConvert_SQL_CONVERT_BIT SqlSupportsConvert = 2 + SqlSupportsConvert_SQL_CONVERT_CHAR SqlSupportsConvert = 3 + SqlSupportsConvert_SQL_CONVERT_DATE SqlSupportsConvert = 4 + SqlSupportsConvert_SQL_CONVERT_DECIMAL SqlSupportsConvert = 5 + SqlSupportsConvert_SQL_CONVERT_FLOAT SqlSupportsConvert = 6 + SqlSupportsConvert_SQL_CONVERT_INTEGER SqlSupportsConvert = 7 + SqlSupportsConvert_SQL_CONVERT_INTERVAL_DAY_TIME SqlSupportsConvert = 8 + SqlSupportsConvert_SQL_CONVERT_INTERVAL_YEAR_MONTH SqlSupportsConvert = 9 + SqlSupportsConvert_SQL_CONVERT_LONGVARBINARY SqlSupportsConvert = 10 + SqlSupportsConvert_SQL_CONVERT_LONGVARCHAR SqlSupportsConvert = 11 + SqlSupportsConvert_SQL_CONVERT_NUMERIC SqlSupportsConvert = 12 + SqlSupportsConvert_SQL_CONVERT_REAL SqlSupportsConvert = 13 + SqlSupportsConvert_SQL_CONVERT_SMALLINT SqlSupportsConvert = 14 + SqlSupportsConvert_SQL_CONVERT_TIME SqlSupportsConvert = 15 + SqlSupportsConvert_SQL_CONVERT_TIMESTAMP SqlSupportsConvert = 16 + SqlSupportsConvert_SQL_CONVERT_TINYINT SqlSupportsConvert = 17 + SqlSupportsConvert_SQL_CONVERT_VARBINARY SqlSupportsConvert = 18 + SqlSupportsConvert_SQL_CONVERT_VARCHAR SqlSupportsConvert = 19 +) + +// Enum value maps for SqlSupportsConvert. +var ( + SqlSupportsConvert_name = map[int32]string{ + 0: "SQL_CONVERT_BIGINT", + 1: "SQL_CONVERT_BINARY", + 2: "SQL_CONVERT_BIT", + 3: "SQL_CONVERT_CHAR", + 4: "SQL_CONVERT_DATE", + 5: "SQL_CONVERT_DECIMAL", + 6: "SQL_CONVERT_FLOAT", + 7: "SQL_CONVERT_INTEGER", + 8: "SQL_CONVERT_INTERVAL_DAY_TIME", + 9: "SQL_CONVERT_INTERVAL_YEAR_MONTH", + 10: "SQL_CONVERT_LONGVARBINARY", + 11: "SQL_CONVERT_LONGVARCHAR", + 12: "SQL_CONVERT_NUMERIC", + 13: "SQL_CONVERT_REAL", + 14: "SQL_CONVERT_SMALLINT", + 15: "SQL_CONVERT_TIME", + 16: "SQL_CONVERT_TIMESTAMP", + 17: "SQL_CONVERT_TINYINT", + 18: "SQL_CONVERT_VARBINARY", + 19: "SQL_CONVERT_VARCHAR", + } + SqlSupportsConvert_value = map[string]int32{ + "SQL_CONVERT_BIGINT": 0, + "SQL_CONVERT_BINARY": 1, + "SQL_CONVERT_BIT": 2, + "SQL_CONVERT_CHAR": 3, + "SQL_CONVERT_DATE": 4, + "SQL_CONVERT_DECIMAL": 5, + "SQL_CONVERT_FLOAT": 6, + "SQL_CONVERT_INTEGER": 7, + "SQL_CONVERT_INTERVAL_DAY_TIME": 8, + "SQL_CONVERT_INTERVAL_YEAR_MONTH": 9, + "SQL_CONVERT_LONGVARBINARY": 10, + "SQL_CONVERT_LONGVARCHAR": 11, + "SQL_CONVERT_NUMERIC": 12, + "SQL_CONVERT_REAL": 13, + "SQL_CONVERT_SMALLINT": 14, + "SQL_CONVERT_TIME": 15, + "SQL_CONVERT_TIMESTAMP": 16, + "SQL_CONVERT_TINYINT": 17, + "SQL_CONVERT_VARBINARY": 18, + "SQL_CONVERT_VARCHAR": 19, + } +) + +func (x SqlSupportsConvert) Enum() *SqlSupportsConvert { + p := new(SqlSupportsConvert) + *p = x + return p +} + +func (x SqlSupportsConvert) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SqlSupportsConvert) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[15].Descriptor() +} + +func (SqlSupportsConvert) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[15] +} + +func (x SqlSupportsConvert) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SqlSupportsConvert.Descriptor instead. +func (SqlSupportsConvert) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{15} +} + +//* +// The JDBC/ODBC-defined type of any object. +// All the values here are the sames as in the JDBC and ODBC specs. +type XdbcDataType int32 + +const ( + XdbcDataType_XDBC_UNKNOWN_TYPE XdbcDataType = 0 + XdbcDataType_XDBC_CHAR XdbcDataType = 1 + XdbcDataType_XDBC_NUMERIC XdbcDataType = 2 + XdbcDataType_XDBC_DECIMAL XdbcDataType = 3 + XdbcDataType_XDBC_INTEGER XdbcDataType = 4 + XdbcDataType_XDBC_SMALLINT XdbcDataType = 5 + XdbcDataType_XDBC_FLOAT XdbcDataType = 6 + XdbcDataType_XDBC_REAL XdbcDataType = 7 + XdbcDataType_XDBC_DOUBLE XdbcDataType = 8 + XdbcDataType_XDBC_DATETIME XdbcDataType = 9 + XdbcDataType_XDBC_INTERVAL XdbcDataType = 10 + XdbcDataType_XDBC_VARCHAR XdbcDataType = 12 + XdbcDataType_XDBC_DATE XdbcDataType = 91 + XdbcDataType_XDBC_TIME XdbcDataType = 92 + XdbcDataType_XDBC_TIMESTAMP XdbcDataType = 93 + XdbcDataType_XDBC_LONGVARCHAR XdbcDataType = -1 + XdbcDataType_XDBC_BINARY XdbcDataType = -2 + XdbcDataType_XDBC_VARBINARY XdbcDataType = -3 + XdbcDataType_XDBC_LONGVARBINARY XdbcDataType = -4 + XdbcDataType_XDBC_BIGINT XdbcDataType = -5 + XdbcDataType_XDBC_TINYINT XdbcDataType = -6 + XdbcDataType_XDBC_BIT XdbcDataType = -7 + XdbcDataType_XDBC_WCHAR XdbcDataType = -8 + XdbcDataType_XDBC_WVARCHAR XdbcDataType = -9 +) + +// Enum value maps for XdbcDataType. +var ( + XdbcDataType_name = map[int32]string{ + 0: "XDBC_UNKNOWN_TYPE", + 1: "XDBC_CHAR", + 2: "XDBC_NUMERIC", + 3: "XDBC_DECIMAL", + 4: "XDBC_INTEGER", + 5: "XDBC_SMALLINT", + 6: "XDBC_FLOAT", + 7: "XDBC_REAL", + 8: "XDBC_DOUBLE", + 9: "XDBC_DATETIME", + 10: "XDBC_INTERVAL", + 12: "XDBC_VARCHAR", + 91: "XDBC_DATE", + 92: "XDBC_TIME", + 93: "XDBC_TIMESTAMP", + -1: "XDBC_LONGVARCHAR", + -2: "XDBC_BINARY", + -3: "XDBC_VARBINARY", + -4: "XDBC_LONGVARBINARY", + -5: "XDBC_BIGINT", + -6: "XDBC_TINYINT", + -7: "XDBC_BIT", + -8: "XDBC_WCHAR", + -9: "XDBC_WVARCHAR", + } + XdbcDataType_value = map[string]int32{ + "XDBC_UNKNOWN_TYPE": 0, + "XDBC_CHAR": 1, + "XDBC_NUMERIC": 2, + "XDBC_DECIMAL": 3, + "XDBC_INTEGER": 4, + "XDBC_SMALLINT": 5, + "XDBC_FLOAT": 6, + "XDBC_REAL": 7, + "XDBC_DOUBLE": 8, + "XDBC_DATETIME": 9, + "XDBC_INTERVAL": 10, + "XDBC_VARCHAR": 12, + "XDBC_DATE": 91, + "XDBC_TIME": 92, + "XDBC_TIMESTAMP": 93, + "XDBC_LONGVARCHAR": -1, + "XDBC_BINARY": -2, + "XDBC_VARBINARY": -3, + "XDBC_LONGVARBINARY": -4, + "XDBC_BIGINT": -5, + "XDBC_TINYINT": -6, + "XDBC_BIT": -7, + "XDBC_WCHAR": -8, + "XDBC_WVARCHAR": -9, + } +) + +func (x XdbcDataType) Enum() *XdbcDataType { + p := new(XdbcDataType) + *p = x + return p +} + +func (x XdbcDataType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (XdbcDataType) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[16].Descriptor() +} + +func (XdbcDataType) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[16] +} + +func (x XdbcDataType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use XdbcDataType.Descriptor instead. +func (XdbcDataType) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{16} +} + +//* +// Detailed subtype information for XDBC_TYPE_DATETIME and XDBC_TYPE_INTERVAL. +type XdbcDatetimeSubcode int32 + +const ( + XdbcDatetimeSubcode_XDBC_SUBCODE_UNKNOWN XdbcDatetimeSubcode = 0 + XdbcDatetimeSubcode_XDBC_SUBCODE_YEAR XdbcDatetimeSubcode = 1 + XdbcDatetimeSubcode_XDBC_SUBCODE_DATE XdbcDatetimeSubcode = 1 + XdbcDatetimeSubcode_XDBC_SUBCODE_TIME XdbcDatetimeSubcode = 2 + XdbcDatetimeSubcode_XDBC_SUBCODE_MONTH XdbcDatetimeSubcode = 2 + XdbcDatetimeSubcode_XDBC_SUBCODE_TIMESTAMP XdbcDatetimeSubcode = 3 + XdbcDatetimeSubcode_XDBC_SUBCODE_DAY XdbcDatetimeSubcode = 3 + XdbcDatetimeSubcode_XDBC_SUBCODE_TIME_WITH_TIMEZONE XdbcDatetimeSubcode = 4 + XdbcDatetimeSubcode_XDBC_SUBCODE_HOUR XdbcDatetimeSubcode = 4 + XdbcDatetimeSubcode_XDBC_SUBCODE_TIMESTAMP_WITH_TIMEZONE XdbcDatetimeSubcode = 5 + XdbcDatetimeSubcode_XDBC_SUBCODE_MINUTE XdbcDatetimeSubcode = 5 + XdbcDatetimeSubcode_XDBC_SUBCODE_SECOND XdbcDatetimeSubcode = 6 + XdbcDatetimeSubcode_XDBC_SUBCODE_YEAR_TO_MONTH XdbcDatetimeSubcode = 7 + XdbcDatetimeSubcode_XDBC_SUBCODE_DAY_TO_HOUR XdbcDatetimeSubcode = 8 + XdbcDatetimeSubcode_XDBC_SUBCODE_DAY_TO_MINUTE XdbcDatetimeSubcode = 9 + XdbcDatetimeSubcode_XDBC_SUBCODE_DAY_TO_SECOND XdbcDatetimeSubcode = 10 + XdbcDatetimeSubcode_XDBC_SUBCODE_HOUR_TO_MINUTE XdbcDatetimeSubcode = 11 + XdbcDatetimeSubcode_XDBC_SUBCODE_HOUR_TO_SECOND XdbcDatetimeSubcode = 12 + XdbcDatetimeSubcode_XDBC_SUBCODE_MINUTE_TO_SECOND XdbcDatetimeSubcode = 13 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_YEAR XdbcDatetimeSubcode = 101 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_MONTH XdbcDatetimeSubcode = 102 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_DAY XdbcDatetimeSubcode = 103 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_HOUR XdbcDatetimeSubcode = 104 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_MINUTE XdbcDatetimeSubcode = 105 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_SECOND XdbcDatetimeSubcode = 106 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_YEAR_TO_MONTH XdbcDatetimeSubcode = 107 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_DAY_TO_HOUR XdbcDatetimeSubcode = 108 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_DAY_TO_MINUTE XdbcDatetimeSubcode = 109 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_DAY_TO_SECOND XdbcDatetimeSubcode = 110 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_HOUR_TO_MINUTE XdbcDatetimeSubcode = 111 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_HOUR_TO_SECOND XdbcDatetimeSubcode = 112 + XdbcDatetimeSubcode_XDBC_SUBCODE_INTERVAL_MINUTE_TO_SECOND XdbcDatetimeSubcode = 113 +) + +// Enum value maps for XdbcDatetimeSubcode. +var ( + XdbcDatetimeSubcode_name = map[int32]string{ + 0: "XDBC_SUBCODE_UNKNOWN", + 1: "XDBC_SUBCODE_YEAR", + // Duplicate value: 1: "XDBC_SUBCODE_DATE", + 2: "XDBC_SUBCODE_TIME", + // Duplicate value: 2: "XDBC_SUBCODE_MONTH", + 3: "XDBC_SUBCODE_TIMESTAMP", + // Duplicate value: 3: "XDBC_SUBCODE_DAY", + 4: "XDBC_SUBCODE_TIME_WITH_TIMEZONE", + // Duplicate value: 4: "XDBC_SUBCODE_HOUR", + 5: "XDBC_SUBCODE_TIMESTAMP_WITH_TIMEZONE", + // Duplicate value: 5: "XDBC_SUBCODE_MINUTE", + 6: "XDBC_SUBCODE_SECOND", + 7: "XDBC_SUBCODE_YEAR_TO_MONTH", + 8: "XDBC_SUBCODE_DAY_TO_HOUR", + 9: "XDBC_SUBCODE_DAY_TO_MINUTE", + 10: "XDBC_SUBCODE_DAY_TO_SECOND", + 11: "XDBC_SUBCODE_HOUR_TO_MINUTE", + 12: "XDBC_SUBCODE_HOUR_TO_SECOND", + 13: "XDBC_SUBCODE_MINUTE_TO_SECOND", + 101: "XDBC_SUBCODE_INTERVAL_YEAR", + 102: "XDBC_SUBCODE_INTERVAL_MONTH", + 103: "XDBC_SUBCODE_INTERVAL_DAY", + 104: "XDBC_SUBCODE_INTERVAL_HOUR", + 105: "XDBC_SUBCODE_INTERVAL_MINUTE", + 106: "XDBC_SUBCODE_INTERVAL_SECOND", + 107: "XDBC_SUBCODE_INTERVAL_YEAR_TO_MONTH", + 108: "XDBC_SUBCODE_INTERVAL_DAY_TO_HOUR", + 109: "XDBC_SUBCODE_INTERVAL_DAY_TO_MINUTE", + 110: "XDBC_SUBCODE_INTERVAL_DAY_TO_SECOND", + 111: "XDBC_SUBCODE_INTERVAL_HOUR_TO_MINUTE", + 112: "XDBC_SUBCODE_INTERVAL_HOUR_TO_SECOND", + 113: "XDBC_SUBCODE_INTERVAL_MINUTE_TO_SECOND", + } + XdbcDatetimeSubcode_value = map[string]int32{ + "XDBC_SUBCODE_UNKNOWN": 0, + "XDBC_SUBCODE_YEAR": 1, + "XDBC_SUBCODE_DATE": 1, + "XDBC_SUBCODE_TIME": 2, + "XDBC_SUBCODE_MONTH": 2, + "XDBC_SUBCODE_TIMESTAMP": 3, + "XDBC_SUBCODE_DAY": 3, + "XDBC_SUBCODE_TIME_WITH_TIMEZONE": 4, + "XDBC_SUBCODE_HOUR": 4, + "XDBC_SUBCODE_TIMESTAMP_WITH_TIMEZONE": 5, + "XDBC_SUBCODE_MINUTE": 5, + "XDBC_SUBCODE_SECOND": 6, + "XDBC_SUBCODE_YEAR_TO_MONTH": 7, + "XDBC_SUBCODE_DAY_TO_HOUR": 8, + "XDBC_SUBCODE_DAY_TO_MINUTE": 9, + "XDBC_SUBCODE_DAY_TO_SECOND": 10, + "XDBC_SUBCODE_HOUR_TO_MINUTE": 11, + "XDBC_SUBCODE_HOUR_TO_SECOND": 12, + "XDBC_SUBCODE_MINUTE_TO_SECOND": 13, + "XDBC_SUBCODE_INTERVAL_YEAR": 101, + "XDBC_SUBCODE_INTERVAL_MONTH": 102, + "XDBC_SUBCODE_INTERVAL_DAY": 103, + "XDBC_SUBCODE_INTERVAL_HOUR": 104, + "XDBC_SUBCODE_INTERVAL_MINUTE": 105, + "XDBC_SUBCODE_INTERVAL_SECOND": 106, + "XDBC_SUBCODE_INTERVAL_YEAR_TO_MONTH": 107, + "XDBC_SUBCODE_INTERVAL_DAY_TO_HOUR": 108, + "XDBC_SUBCODE_INTERVAL_DAY_TO_MINUTE": 109, + "XDBC_SUBCODE_INTERVAL_DAY_TO_SECOND": 110, + "XDBC_SUBCODE_INTERVAL_HOUR_TO_MINUTE": 111, + "XDBC_SUBCODE_INTERVAL_HOUR_TO_SECOND": 112, + "XDBC_SUBCODE_INTERVAL_MINUTE_TO_SECOND": 113, + } +) + +func (x XdbcDatetimeSubcode) Enum() *XdbcDatetimeSubcode { + p := new(XdbcDatetimeSubcode) + *p = x + return p +} + +func (x XdbcDatetimeSubcode) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (XdbcDatetimeSubcode) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[17].Descriptor() +} + +func (XdbcDatetimeSubcode) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[17] +} + +func (x XdbcDatetimeSubcode) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use XdbcDatetimeSubcode.Descriptor instead. +func (XdbcDatetimeSubcode) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{17} +} + +type Nullable int32 + +const ( + //* + // Indicates that the fields does not allow the use of null values. + Nullable_NULLABILITY_NO_NULLS Nullable = 0 + //* + // Indicates that the fields allow the use of null values. + Nullable_NULLABILITY_NULLABLE Nullable = 1 + //* + // Indicates that nullability of the fields can not be determined. + Nullable_NULLABILITY_UNKNOWN Nullable = 2 +) + +// Enum value maps for Nullable. +var ( + Nullable_name = map[int32]string{ + 0: "NULLABILITY_NO_NULLS", + 1: "NULLABILITY_NULLABLE", + 2: "NULLABILITY_UNKNOWN", + } + Nullable_value = map[string]int32{ + "NULLABILITY_NO_NULLS": 0, + "NULLABILITY_NULLABLE": 1, + "NULLABILITY_UNKNOWN": 2, + } +) + +func (x Nullable) Enum() *Nullable { + p := new(Nullable) + *p = x + return p +} + +func (x Nullable) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (Nullable) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[18].Descriptor() +} + +func (Nullable) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[18] +} + +func (x Nullable) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use Nullable.Descriptor instead. +func (Nullable) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{18} +} + +type Searchable int32 + +const ( + //* + // Indicates that column can not be used in a WHERE clause. + Searchable_SEARCHABLE_NONE Searchable = 0 + //* + // Indicates that the column can be used in a WHERE clause if it is using a + // LIKE operator. + Searchable_SEARCHABLE_CHAR Searchable = 1 + //* + // Indicates that the column can be used In a WHERE clause with any + // operator other than LIKE. + // + // - Allowed operators: comparison, quantified comparison, BETWEEN, + // DISTINCT, IN, MATCH, and UNIQUE. + Searchable_SEARCHABLE_BASIC Searchable = 2 + //* + // Indicates that the column can be used in a WHERE clause using any operator. + Searchable_SEARCHABLE_FULL Searchable = 3 +) + +// Enum value maps for Searchable. +var ( + Searchable_name = map[int32]string{ + 0: "SEARCHABLE_NONE", + 1: "SEARCHABLE_CHAR", + 2: "SEARCHABLE_BASIC", + 3: "SEARCHABLE_FULL", + } + Searchable_value = map[string]int32{ + "SEARCHABLE_NONE": 0, + "SEARCHABLE_CHAR": 1, + "SEARCHABLE_BASIC": 2, + "SEARCHABLE_FULL": 3, + } +) + +func (x Searchable) Enum() *Searchable { + p := new(Searchable) + *p = x + return p +} + +func (x Searchable) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (Searchable) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[19].Descriptor() +} + +func (Searchable) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[19] +} + +func (x Searchable) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use Searchable.Descriptor instead. +func (Searchable) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{19} +} + +type UpdateDeleteRules int32 + +const ( + UpdateDeleteRules_CASCADE UpdateDeleteRules = 0 + UpdateDeleteRules_RESTRICT UpdateDeleteRules = 1 + UpdateDeleteRules_SET_NULL UpdateDeleteRules = 2 + UpdateDeleteRules_NO_ACTION UpdateDeleteRules = 3 + UpdateDeleteRules_SET_DEFAULT UpdateDeleteRules = 4 +) + +// Enum value maps for UpdateDeleteRules. +var ( + UpdateDeleteRules_name = map[int32]string{ + 0: "CASCADE", + 1: "RESTRICT", + 2: "SET_NULL", + 3: "NO_ACTION", + 4: "SET_DEFAULT", + } + UpdateDeleteRules_value = map[string]int32{ + "CASCADE": 0, + "RESTRICT": 1, + "SET_NULL": 2, + "NO_ACTION": 3, + "SET_DEFAULT": 4, + } +) + +func (x UpdateDeleteRules) Enum() *UpdateDeleteRules { + p := new(UpdateDeleteRules) + *p = x + return p +} + +func (x UpdateDeleteRules) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (UpdateDeleteRules) Descriptor() protoreflect.EnumDescriptor { + return file_FlightSql_proto_enumTypes[20].Descriptor() +} + +func (UpdateDeleteRules) Type() protoreflect.EnumType { + return &file_FlightSql_proto_enumTypes[20] +} + +func (x UpdateDeleteRules) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use UpdateDeleteRules.Descriptor instead. +func (UpdateDeleteRules) EnumDescriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{20} +} + +// +// Represents a metadata request. Used in the command member of FlightDescriptor +// for the following RPC calls: +// - GetSchema: return the Arrow schema of the query. +// - GetFlightInfo: execute the metadata request. +// +// The returned Arrow schema will be: +// < +// info_name: uint32 not null, +// value: dense_union< +// string_value: utf8, +// bool_value: bool, +// bigint_value: int64, +// int32_bitmask: int32, +// string_list: list +// int32_to_int32_list_map: map> +// > +// where there is one row per requested piece of metadata information. +type CommandGetSqlInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // + // Values are modelled after ODBC's SQLGetInfo() function. This information is intended to provide + // Flight SQL clients with basic, SQL syntax and SQL functions related information. + // More information types can be added in future releases. + // E.g. more SQL syntax support types, scalar functions support, type conversion support etc. + // + // Note that the set of metadata may expand. + // + // Initially, Flight SQL will support the following information types: + // - Server Information - Range [0-500) + // - Syntax Information - Range [500-1000) + // Range [0-10,000) is reserved for defaults (see SqlInfo enum for default options). + // Custom options should start at 10,000. + // + // If omitted, then all metadata will be retrieved. + // Flight SQL Servers may choose to include additional metadata above and beyond the specified set, however they must + // at least return the specified set. IDs ranging from 0 to 10,000 (exclusive) are reserved for future use. + // If additional metadata is included, the metadata IDs should start from 10,000. + Info []uint32 `protobuf:"varint,1,rep,packed,name=info,proto3" json:"info,omitempty"` +} + +func (x *CommandGetSqlInfo) Reset() { + *x = CommandGetSqlInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandGetSqlInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandGetSqlInfo) ProtoMessage() {} + +func (x *CommandGetSqlInfo) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[0] + 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 CommandGetSqlInfo.ProtoReflect.Descriptor instead. +func (*CommandGetSqlInfo) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{0} +} + +func (x *CommandGetSqlInfo) GetInfo() []uint32 { + if x != nil { + return x.Info + } + return nil +} + +// +// Represents a request to retrieve information about data type supported on a Flight SQL enabled backend. +// Used in the command member of FlightDescriptor for the following RPC calls: +// - GetSchema: return the schema of the query. +// - GetFlightInfo: execute the catalog metadata request. +// +// The returned schema will be: +// < +// type_name: utf8 not null (The name of the data type, for example: VARCHAR, INTEGER, etc), +// data_type: int not null (The SQL data type), +// column_size: int (The maximum size supported by that column. +// In case of exact numeric types, this represents the maximum precision. +// In case of string types, this represents the character length. +// In case of datetime data types, this represents the length in characters of the string representation. +// NULL is returned for data types where column size is not applicable.), +// literal_prefix: utf8 (Character or characters used to prefix a literal, NULL is returned for +// data types where a literal prefix is not applicable.), +// literal_suffix: utf8 (Character or characters used to terminate a literal, +// NULL is returned for data types where a literal suffix is not applicable.), +// create_params: list +// (A list of keywords corresponding to which parameters can be used when creating +// a column for that specific type. +// NULL is returned if there are no parameters for the data type definition.), +// nullable: int not null (Shows if the data type accepts a NULL value. The possible values can be seen in the +// Nullable enum.), +// case_sensitive: bool not null (Shows if a character data type is case-sensitive in collations and comparisons), +// searchable: int not null (Shows how the data type is used in a WHERE clause. The possible values can be seen in the +// Searchable enum.), +// unsigned_attribute: bool (Shows if the data type is unsigned. NULL is returned if the attribute is +// not applicable to the data type or the data type is not numeric.), +// fixed_prec_scale: bool not null (Shows if the data type has predefined fixed precision and scale.), +// auto_increment: bool (Shows if the data type is auto incremental. NULL is returned if the attribute +// is not applicable to the data type or the data type is not numeric.), +// local_type_name: utf8 (Localized version of the data source-dependent name of the data type. NULL +// is returned if a localized name is not supported by the data source), +// minimum_scale: int (The minimum scale of the data type on the data source. +// If a data type has a fixed scale, the MINIMUM_SCALE and MAXIMUM_SCALE +// columns both contain this value. NULL is returned if scale is not applicable.), +// maximum_scale: int (The maximum scale of the data type on the data source. +// NULL is returned if scale is not applicable.), +// sql_data_type: int not null (The value of the SQL DATA TYPE which has the same values +// as data_type value. Except for interval and datetime, which +// uses generic values. More info about those types can be +// obtained through datetime_subcode. The possible values can be seen +// in the XdbcDataType enum.), +// datetime_subcode: int (Only used when the SQL DATA TYPE is interval or datetime. It contains +// its sub types. For type different from interval and datetime, this value +// is NULL. The possible values can be seen in the XdbcDatetimeSubcode enum.), +// num_prec_radix: int (If the data type is an approximate numeric type, this column contains +// the value 2 to indicate that COLUMN_SIZE specifies a number of bits. For +// exact numeric types, this column contains the value 10 to indicate that +// column size specifies a number of decimal digits. Otherwise, this column is NULL.), +// interval_precision: int (If the data type is an interval data type, then this column contains the value +// of the interval leading precision. Otherwise, this column is NULL. This fields +// is only relevant to be used by ODBC). +// > +// The returned data should be ordered by data_type and then by type_name. +type CommandGetXdbcTypeInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // + // Specifies the data type to search for the info. + DataType *int32 `protobuf:"varint,1,opt,name=data_type,json=dataType,proto3,oneof" json:"data_type,omitempty"` +} + +func (x *CommandGetXdbcTypeInfo) Reset() { + *x = CommandGetXdbcTypeInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandGetXdbcTypeInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandGetXdbcTypeInfo) ProtoMessage() {} + +func (x *CommandGetXdbcTypeInfo) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[1] + 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 CommandGetXdbcTypeInfo.ProtoReflect.Descriptor instead. +func (*CommandGetXdbcTypeInfo) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{1} +} + +func (x *CommandGetXdbcTypeInfo) GetDataType() int32 { + if x != nil && x.DataType != nil { + return *x.DataType + } + return 0 +} + +// +// Represents a request to retrieve the list of catalogs on a Flight SQL enabled backend. +// The definition of a catalog depends on vendor/implementation. It is usually the database itself +// Used in the command member of FlightDescriptor for the following RPC calls: +// - GetSchema: return the Arrow schema of the query. +// - GetFlightInfo: execute the catalog metadata request. +// +// The returned Arrow schema will be: +// < +// catalog_name: utf8 not null +// > +// The returned data should be ordered by catalog_name. +type CommandGetCatalogs struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CommandGetCatalogs) Reset() { + *x = CommandGetCatalogs{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandGetCatalogs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandGetCatalogs) ProtoMessage() {} + +func (x *CommandGetCatalogs) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[2] + 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 CommandGetCatalogs.ProtoReflect.Descriptor instead. +func (*CommandGetCatalogs) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{2} +} + +// +// Represents a request to retrieve the list of database schemas on a Flight SQL enabled backend. +// The definition of a database schema depends on vendor/implementation. It is usually a collection of tables. +// Used in the command member of FlightDescriptor for the following RPC calls: +// - GetSchema: return the Arrow schema of the query. +// - GetFlightInfo: execute the catalog metadata request. +// +// The returned Arrow schema will be: +// < +// catalog_name: utf8, +// db_schema_name: utf8 not null +// > +// The returned data should be ordered by catalog_name, then db_schema_name. +type CommandGetDbSchemas struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // + // Specifies the Catalog to search for the tables. + // An empty string retrieves those without a catalog. + // If omitted the catalog name should not be used to narrow the search. + Catalog *string `protobuf:"bytes,1,opt,name=catalog,proto3,oneof" json:"catalog,omitempty"` + // + // Specifies a filter pattern for schemas to search for. + // When no db_schema_filter_pattern is provided, the pattern will not be used to narrow the search. + // In the pattern string, two special characters can be used to denote matching rules: + // - "%" means to match any substring with 0 or more characters. + // - "_" means to match any one character. + DbSchemaFilterPattern *string `protobuf:"bytes,2,opt,name=db_schema_filter_pattern,json=dbSchemaFilterPattern,proto3,oneof" json:"db_schema_filter_pattern,omitempty"` +} + +func (x *CommandGetDbSchemas) Reset() { + *x = CommandGetDbSchemas{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandGetDbSchemas) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandGetDbSchemas) ProtoMessage() {} + +func (x *CommandGetDbSchemas) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[3] + 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 CommandGetDbSchemas.ProtoReflect.Descriptor instead. +func (*CommandGetDbSchemas) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{3} +} + +func (x *CommandGetDbSchemas) GetCatalog() string { + if x != nil && x.Catalog != nil { + return *x.Catalog + } + return "" +} + +func (x *CommandGetDbSchemas) GetDbSchemaFilterPattern() string { + if x != nil && x.DbSchemaFilterPattern != nil { + return *x.DbSchemaFilterPattern + } + return "" +} + +// +// Represents a request to retrieve the list of tables, and optionally their schemas, on a Flight SQL enabled backend. +// Used in the command member of FlightDescriptor for the following RPC calls: +// - GetSchema: return the Arrow schema of the query. +// - GetFlightInfo: execute the catalog metadata request. +// +// The returned Arrow schema will be: +// < +// catalog_name: utf8, +// db_schema_name: utf8, +// table_name: utf8 not null, +// table_type: utf8 not null, +// [optional] table_schema: bytes not null (schema of the table as described in Schema.fbs::Schema, +// it is serialized as an IPC message.) +// > +// Fields on table_schema may contain the following metadata: +// - ARROW:FLIGHT:SQL:CATALOG_NAME - Table's catalog name +// - ARROW:FLIGHT:SQL:DB_SCHEMA_NAME - Database schema name +// - ARROW:FLIGHT:SQL:TABLE_NAME - Table name +// - ARROW:FLIGHT:SQL:TYPE_NAME - The data source-specific name for the data type of the column. +// - ARROW:FLIGHT:SQL:PRECISION - Column precision/size +// - ARROW:FLIGHT:SQL:SCALE - Column scale/decimal digits if applicable +// - ARROW:FLIGHT:SQL:IS_AUTO_INCREMENT - "1" indicates if the column is auto incremented, "0" otherwise. +// - ARROW:FLIGHT:SQL:IS_CASE_SENSITIVE - "1" indicates if the column is case sensitive, "0" otherwise. +// - ARROW:FLIGHT:SQL:IS_READ_ONLY - "1" indicates if the column is read only, "0" otherwise. +// - ARROW:FLIGHT:SQL:IS_SEARCHABLE - "1" indicates if the column is searchable via WHERE clause, "0" otherwise. +// The returned data should be ordered by catalog_name, db_schema_name, table_name, then table_type, followed by table_schema if requested. +type CommandGetTables struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // + // Specifies the Catalog to search for the tables. + // An empty string retrieves those without a catalog. + // If omitted the catalog name should not be used to narrow the search. + Catalog *string `protobuf:"bytes,1,opt,name=catalog,proto3,oneof" json:"catalog,omitempty"` + // + // Specifies a filter pattern for schemas to search for. + // When no db_schema_filter_pattern is provided, all schemas matching other filters are searched. + // In the pattern string, two special characters can be used to denote matching rules: + // - "%" means to match any substring with 0 or more characters. + // - "_" means to match any one character. + DbSchemaFilterPattern *string `protobuf:"bytes,2,opt,name=db_schema_filter_pattern,json=dbSchemaFilterPattern,proto3,oneof" json:"db_schema_filter_pattern,omitempty"` + // + // Specifies a filter pattern for tables to search for. + // When no table_name_filter_pattern is provided, all tables matching other filters are searched. + // In the pattern string, two special characters can be used to denote matching rules: + // - "%" means to match any substring with 0 or more characters. + // - "_" means to match any one character. + TableNameFilterPattern *string `protobuf:"bytes,3,opt,name=table_name_filter_pattern,json=tableNameFilterPattern,proto3,oneof" json:"table_name_filter_pattern,omitempty"` + // + // Specifies a filter of table types which must match. + // The table types depend on vendor/implementation. It is usually used to separate tables from views or system tables. + // TABLE, VIEW, and SYSTEM TABLE are commonly supported. + TableTypes []string `protobuf:"bytes,4,rep,name=table_types,json=tableTypes,proto3" json:"table_types,omitempty"` + // Specifies if the Arrow schema should be returned for found tables. + IncludeSchema bool `protobuf:"varint,5,opt,name=include_schema,json=includeSchema,proto3" json:"include_schema,omitempty"` +} + +func (x *CommandGetTables) Reset() { + *x = CommandGetTables{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandGetTables) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandGetTables) ProtoMessage() {} + +func (x *CommandGetTables) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[4] + 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 CommandGetTables.ProtoReflect.Descriptor instead. +func (*CommandGetTables) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{4} +} + +func (x *CommandGetTables) GetCatalog() string { + if x != nil && x.Catalog != nil { + return *x.Catalog + } + return "" +} + +func (x *CommandGetTables) GetDbSchemaFilterPattern() string { + if x != nil && x.DbSchemaFilterPattern != nil { + return *x.DbSchemaFilterPattern + } + return "" +} + +func (x *CommandGetTables) GetTableNameFilterPattern() string { + if x != nil && x.TableNameFilterPattern != nil { + return *x.TableNameFilterPattern + } + return "" +} + +func (x *CommandGetTables) GetTableTypes() []string { + if x != nil { + return x.TableTypes + } + return nil +} + +func (x *CommandGetTables) GetIncludeSchema() bool { + if x != nil { + return x.IncludeSchema + } + return false +} + +// +// Represents a request to retrieve the list of table types on a Flight SQL enabled backend. +// The table types depend on vendor/implementation. It is usually used to separate tables from views or system tables. +// TABLE, VIEW, and SYSTEM TABLE are commonly supported. +// Used in the command member of FlightDescriptor for the following RPC calls: +// - GetSchema: return the Arrow schema of the query. +// - GetFlightInfo: execute the catalog metadata request. +// +// The returned Arrow schema will be: +// < +// table_type: utf8 not null +// > +// The returned data should be ordered by table_type. +type CommandGetTableTypes struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *CommandGetTableTypes) Reset() { + *x = CommandGetTableTypes{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandGetTableTypes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandGetTableTypes) ProtoMessage() {} + +func (x *CommandGetTableTypes) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[5] + 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 CommandGetTableTypes.ProtoReflect.Descriptor instead. +func (*CommandGetTableTypes) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{5} +} + +// +// Represents a request to retrieve the primary keys of a table on a Flight SQL enabled backend. +// Used in the command member of FlightDescriptor for the following RPC calls: +// - GetSchema: return the Arrow schema of the query. +// - GetFlightInfo: execute the catalog metadata request. +// +// The returned Arrow schema will be: +// < +// catalog_name: utf8, +// db_schema_name: utf8, +// table_name: utf8 not null, +// column_name: utf8 not null, +// key_name: utf8, +// key_sequence: int not null +// > +// The returned data should be ordered by catalog_name, db_schema_name, table_name, key_name, then key_sequence. +type CommandGetPrimaryKeys struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // + // Specifies the catalog to search for the table. + // An empty string retrieves those without a catalog. + // If omitted the catalog name should not be used to narrow the search. + Catalog *string `protobuf:"bytes,1,opt,name=catalog,proto3,oneof" json:"catalog,omitempty"` + // + // Specifies the schema to search for the table. + // An empty string retrieves those without a schema. + // If omitted the schema name should not be used to narrow the search. + DbSchema *string `protobuf:"bytes,2,opt,name=db_schema,json=dbSchema,proto3,oneof" json:"db_schema,omitempty"` + // Specifies the table to get the primary keys for. + Table string `protobuf:"bytes,3,opt,name=table,proto3" json:"table,omitempty"` +} + +func (x *CommandGetPrimaryKeys) Reset() { + *x = CommandGetPrimaryKeys{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandGetPrimaryKeys) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandGetPrimaryKeys) ProtoMessage() {} + +func (x *CommandGetPrimaryKeys) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[6] + 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 CommandGetPrimaryKeys.ProtoReflect.Descriptor instead. +func (*CommandGetPrimaryKeys) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{6} +} + +func (x *CommandGetPrimaryKeys) GetCatalog() string { + if x != nil && x.Catalog != nil { + return *x.Catalog + } + return "" +} + +func (x *CommandGetPrimaryKeys) GetDbSchema() string { + if x != nil && x.DbSchema != nil { + return *x.DbSchema + } + return "" +} + +func (x *CommandGetPrimaryKeys) GetTable() string { + if x != nil { + return x.Table + } + return "" +} + +// +// Represents a request to retrieve a description of the foreign key columns that reference the given table's +// primary key columns (the foreign keys exported by a table) of a table on a Flight SQL enabled backend. +// Used in the command member of FlightDescriptor for the following RPC calls: +// - GetSchema: return the Arrow schema of the query. +// - GetFlightInfo: execute the catalog metadata request. +// +// The returned Arrow schema will be: +// < +// pk_catalog_name: utf8, +// pk_db_schema_name: utf8, +// pk_table_name: utf8 not null, +// pk_column_name: utf8 not null, +// fk_catalog_name: utf8, +// fk_db_schema_name: utf8, +// fk_table_name: utf8 not null, +// fk_column_name: utf8 not null, +// key_sequence: int not null, +// fk_key_name: utf8, +// pk_key_name: utf8, +// update_rule: uint1 not null, +// delete_rule: uint1 not null +// > +// The returned data should be ordered by fk_catalog_name, fk_db_schema_name, fk_table_name, fk_key_name, then key_sequence. +// update_rule and delete_rule returns a byte that is equivalent to actions declared on UpdateDeleteRules enum. +type CommandGetExportedKeys struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // + // Specifies the catalog to search for the foreign key table. + // An empty string retrieves those without a catalog. + // If omitted the catalog name should not be used to narrow the search. + Catalog *string `protobuf:"bytes,1,opt,name=catalog,proto3,oneof" json:"catalog,omitempty"` + // + // Specifies the schema to search for the foreign key table. + // An empty string retrieves those without a schema. + // If omitted the schema name should not be used to narrow the search. + DbSchema *string `protobuf:"bytes,2,opt,name=db_schema,json=dbSchema,proto3,oneof" json:"db_schema,omitempty"` + // Specifies the foreign key table to get the foreign keys for. + Table string `protobuf:"bytes,3,opt,name=table,proto3" json:"table,omitempty"` +} + +func (x *CommandGetExportedKeys) Reset() { + *x = CommandGetExportedKeys{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandGetExportedKeys) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandGetExportedKeys) ProtoMessage() {} + +func (x *CommandGetExportedKeys) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[7] + 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 CommandGetExportedKeys.ProtoReflect.Descriptor instead. +func (*CommandGetExportedKeys) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{7} +} + +func (x *CommandGetExportedKeys) GetCatalog() string { + if x != nil && x.Catalog != nil { + return *x.Catalog + } + return "" +} + +func (x *CommandGetExportedKeys) GetDbSchema() string { + if x != nil && x.DbSchema != nil { + return *x.DbSchema + } + return "" +} + +func (x *CommandGetExportedKeys) GetTable() string { + if x != nil { + return x.Table + } + return "" +} + +// +// Represents a request to retrieve the foreign keys of a table on a Flight SQL enabled backend. +// Used in the command member of FlightDescriptor for the following RPC calls: +// - GetSchema: return the Arrow schema of the query. +// - GetFlightInfo: execute the catalog metadata request. +// +// The returned Arrow schema will be: +// < +// pk_catalog_name: utf8, +// pk_db_schema_name: utf8, +// pk_table_name: utf8 not null, +// pk_column_name: utf8 not null, +// fk_catalog_name: utf8, +// fk_db_schema_name: utf8, +// fk_table_name: utf8 not null, +// fk_column_name: utf8 not null, +// key_sequence: int not null, +// fk_key_name: utf8, +// pk_key_name: utf8, +// update_rule: uint1 not null, +// delete_rule: uint1 not null +// > +// The returned data should be ordered by pk_catalog_name, pk_db_schema_name, pk_table_name, pk_key_name, then key_sequence. +// update_rule and delete_rule returns a byte that is equivalent to actions: +// - 0 = CASCADE +// - 1 = RESTRICT +// - 2 = SET NULL +// - 3 = NO ACTION +// - 4 = SET DEFAULT +type CommandGetImportedKeys struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // + // Specifies the catalog to search for the primary key table. + // An empty string retrieves those without a catalog. + // If omitted the catalog name should not be used to narrow the search. + Catalog *string `protobuf:"bytes,1,opt,name=catalog,proto3,oneof" json:"catalog,omitempty"` + // + // Specifies the schema to search for the primary key table. + // An empty string retrieves those without a schema. + // If omitted the schema name should not be used to narrow the search. + DbSchema *string `protobuf:"bytes,2,opt,name=db_schema,json=dbSchema,proto3,oneof" json:"db_schema,omitempty"` + // Specifies the primary key table to get the foreign keys for. + Table string `protobuf:"bytes,3,opt,name=table,proto3" json:"table,omitempty"` +} + +func (x *CommandGetImportedKeys) Reset() { + *x = CommandGetImportedKeys{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandGetImportedKeys) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandGetImportedKeys) ProtoMessage() {} + +func (x *CommandGetImportedKeys) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[8] + 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 CommandGetImportedKeys.ProtoReflect.Descriptor instead. +func (*CommandGetImportedKeys) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{8} +} + +func (x *CommandGetImportedKeys) GetCatalog() string { + if x != nil && x.Catalog != nil { + return *x.Catalog + } + return "" +} + +func (x *CommandGetImportedKeys) GetDbSchema() string { + if x != nil && x.DbSchema != nil { + return *x.DbSchema + } + return "" +} + +func (x *CommandGetImportedKeys) GetTable() string { + if x != nil { + return x.Table + } + return "" +} + +// +// Represents a request to retrieve a description of the foreign key columns in the given foreign key table that +// reference the primary key or the columns representing a unique constraint of the parent table (could be the same +// or a different table) on a Flight SQL enabled backend. +// Used in the command member of FlightDescriptor for the following RPC calls: +// - GetSchema: return the Arrow schema of the query. +// - GetFlightInfo: execute the catalog metadata request. +// +// The returned Arrow schema will be: +// < +// pk_catalog_name: utf8, +// pk_db_schema_name: utf8, +// pk_table_name: utf8 not null, +// pk_column_name: utf8 not null, +// fk_catalog_name: utf8, +// fk_db_schema_name: utf8, +// fk_table_name: utf8 not null, +// fk_column_name: utf8 not null, +// key_sequence: int not null, +// fk_key_name: utf8, +// pk_key_name: utf8, +// update_rule: uint1 not null, +// delete_rule: uint1 not null +// > +// The returned data should be ordered by pk_catalog_name, pk_db_schema_name, pk_table_name, pk_key_name, then key_sequence. +// update_rule and delete_rule returns a byte that is equivalent to actions: +// - 0 = CASCADE +// - 1 = RESTRICT +// - 2 = SET NULL +// - 3 = NO ACTION +// - 4 = SET DEFAULT +type CommandGetCrossReference struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + //* + // The catalog name where the parent table is. + // An empty string retrieves those without a catalog. + // If omitted the catalog name should not be used to narrow the search. + PkCatalog *string `protobuf:"bytes,1,opt,name=pk_catalog,json=pkCatalog,proto3,oneof" json:"pk_catalog,omitempty"` + //* + // The Schema name where the parent table is. + // An empty string retrieves those without a schema. + // If omitted the schema name should not be used to narrow the search. + PkDbSchema *string `protobuf:"bytes,2,opt,name=pk_db_schema,json=pkDbSchema,proto3,oneof" json:"pk_db_schema,omitempty"` + //* + // The parent table name. It cannot be null. + PkTable string `protobuf:"bytes,3,opt,name=pk_table,json=pkTable,proto3" json:"pk_table,omitempty"` + //* + // The catalog name where the foreign table is. + // An empty string retrieves those without a catalog. + // If omitted the catalog name should not be used to narrow the search. + FkCatalog *string `protobuf:"bytes,4,opt,name=fk_catalog,json=fkCatalog,proto3,oneof" json:"fk_catalog,omitempty"` + //* + // The schema name where the foreign table is. + // An empty string retrieves those without a schema. + // If omitted the schema name should not be used to narrow the search. + FkDbSchema *string `protobuf:"bytes,5,opt,name=fk_db_schema,json=fkDbSchema,proto3,oneof" json:"fk_db_schema,omitempty"` + //* + // The foreign table name. It cannot be null. + FkTable string `protobuf:"bytes,6,opt,name=fk_table,json=fkTable,proto3" json:"fk_table,omitempty"` +} + +func (x *CommandGetCrossReference) Reset() { + *x = CommandGetCrossReference{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandGetCrossReference) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandGetCrossReference) ProtoMessage() {} + +func (x *CommandGetCrossReference) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[9] + 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 CommandGetCrossReference.ProtoReflect.Descriptor instead. +func (*CommandGetCrossReference) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{9} +} + +func (x *CommandGetCrossReference) GetPkCatalog() string { + if x != nil && x.PkCatalog != nil { + return *x.PkCatalog + } + return "" +} + +func (x *CommandGetCrossReference) GetPkDbSchema() string { + if x != nil && x.PkDbSchema != nil { + return *x.PkDbSchema + } + return "" +} + +func (x *CommandGetCrossReference) GetPkTable() string { + if x != nil { + return x.PkTable + } + return "" +} + +func (x *CommandGetCrossReference) GetFkCatalog() string { + if x != nil && x.FkCatalog != nil { + return *x.FkCatalog + } + return "" +} + +func (x *CommandGetCrossReference) GetFkDbSchema() string { + if x != nil && x.FkDbSchema != nil { + return *x.FkDbSchema + } + return "" +} + +func (x *CommandGetCrossReference) GetFkTable() string { + if x != nil { + return x.FkTable + } + return "" +} + +// +// Request message for the "CreatePreparedStatement" action on a Flight SQL enabled backend. +type ActionCreatePreparedStatementRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The valid SQL string to create a prepared statement for. + Query string `protobuf:"bytes,1,opt,name=query,proto3" json:"query,omitempty"` +} + +func (x *ActionCreatePreparedStatementRequest) Reset() { + *x = ActionCreatePreparedStatementRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActionCreatePreparedStatementRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActionCreatePreparedStatementRequest) ProtoMessage() {} + +func (x *ActionCreatePreparedStatementRequest) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[10] + 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 ActionCreatePreparedStatementRequest.ProtoReflect.Descriptor instead. +func (*ActionCreatePreparedStatementRequest) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{10} +} + +func (x *ActionCreatePreparedStatementRequest) GetQuery() string { + if x != nil { + return x.Query + } + return "" +} + +// +// Wrap the result of a "GetPreparedStatement" action. +// +// The resultant PreparedStatement can be closed either: +// - Manually, through the "ClosePreparedStatement" action; +// - Automatically, by a server timeout. +type ActionCreatePreparedStatementResult struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Opaque handle for the prepared statement on the server. + PreparedStatementHandle []byte `protobuf:"bytes,1,opt,name=prepared_statement_handle,json=preparedStatementHandle,proto3" json:"prepared_statement_handle,omitempty"` + // If a result set generating query was provided, dataset_schema contains the + // schema of the dataset as described in Schema.fbs::Schema, it is serialized as an IPC message. + DatasetSchema []byte `protobuf:"bytes,2,opt,name=dataset_schema,json=datasetSchema,proto3" json:"dataset_schema,omitempty"` + // If the query provided contained parameters, parameter_schema contains the + // schema of the expected parameters as described in Schema.fbs::Schema, it is serialized as an IPC message. + ParameterSchema []byte `protobuf:"bytes,3,opt,name=parameter_schema,json=parameterSchema,proto3" json:"parameter_schema,omitempty"` +} + +func (x *ActionCreatePreparedStatementResult) Reset() { + *x = ActionCreatePreparedStatementResult{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActionCreatePreparedStatementResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActionCreatePreparedStatementResult) ProtoMessage() {} + +func (x *ActionCreatePreparedStatementResult) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[11] + 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 ActionCreatePreparedStatementResult.ProtoReflect.Descriptor instead. +func (*ActionCreatePreparedStatementResult) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{11} +} + +func (x *ActionCreatePreparedStatementResult) GetPreparedStatementHandle() []byte { + if x != nil { + return x.PreparedStatementHandle + } + return nil +} + +func (x *ActionCreatePreparedStatementResult) GetDatasetSchema() []byte { + if x != nil { + return x.DatasetSchema + } + return nil +} + +func (x *ActionCreatePreparedStatementResult) GetParameterSchema() []byte { + if x != nil { + return x.ParameterSchema + } + return nil +} + +// +// Request message for the "ClosePreparedStatement" action on a Flight SQL enabled backend. +// Closes server resources associated with the prepared statement handle. +type ActionClosePreparedStatementRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Opaque handle for the prepared statement on the server. + PreparedStatementHandle []byte `protobuf:"bytes,1,opt,name=prepared_statement_handle,json=preparedStatementHandle,proto3" json:"prepared_statement_handle,omitempty"` +} + +func (x *ActionClosePreparedStatementRequest) Reset() { + *x = ActionClosePreparedStatementRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ActionClosePreparedStatementRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActionClosePreparedStatementRequest) ProtoMessage() {} + +func (x *ActionClosePreparedStatementRequest) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[12] + 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 ActionClosePreparedStatementRequest.ProtoReflect.Descriptor instead. +func (*ActionClosePreparedStatementRequest) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{12} +} + +func (x *ActionClosePreparedStatementRequest) GetPreparedStatementHandle() []byte { + if x != nil { + return x.PreparedStatementHandle + } + return nil +} + +// +// Represents a SQL query. Used in the command member of FlightDescriptor +// for the following RPC calls: +// - GetSchema: return the Arrow schema of the query. +// Fields on this schema may contain the following metadata: +// - ARROW:FLIGHT:SQL:CATALOG_NAME - Table's catalog name +// - ARROW:FLIGHT:SQL:DB_SCHEMA_NAME - Database schema name +// - ARROW:FLIGHT:SQL:TABLE_NAME - Table name +// - ARROW:FLIGHT:SQL:TYPE_NAME - The data source-specific name for the data type of the column. +// - ARROW:FLIGHT:SQL:PRECISION - Column precision/size +// - ARROW:FLIGHT:SQL:SCALE - Column scale/decimal digits if applicable +// - ARROW:FLIGHT:SQL:IS_AUTO_INCREMENT - "1" indicates if the column is auto incremented, "0" otherwise. +// - ARROW:FLIGHT:SQL:IS_CASE_SENSITIVE - "1" indicates if the column is case sensitive, "0" otherwise. +// - ARROW:FLIGHT:SQL:IS_READ_ONLY - "1" indicates if the column is read only, "0" otherwise. +// - ARROW:FLIGHT:SQL:IS_SEARCHABLE - "1" indicates if the column is searchable via WHERE clause, "0" otherwise. +// - GetFlightInfo: execute the query. +type CommandStatementQuery struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The SQL syntax. + Query string `protobuf:"bytes,1,opt,name=query,proto3" json:"query,omitempty"` +} + +func (x *CommandStatementQuery) Reset() { + *x = CommandStatementQuery{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandStatementQuery) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandStatementQuery) ProtoMessage() {} + +func (x *CommandStatementQuery) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[13] + 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 CommandStatementQuery.ProtoReflect.Descriptor instead. +func (*CommandStatementQuery) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{13} +} + +func (x *CommandStatementQuery) GetQuery() string { + if x != nil { + return x.Query + } + return "" +} + +//* +// Represents a ticket resulting from GetFlightInfo with a CommandStatementQuery. +// This should be used only once and treated as an opaque value, that is, clients should not attempt to parse this. +type TicketStatementQuery struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Unique identifier for the instance of the statement to execute. + StatementHandle []byte `protobuf:"bytes,1,opt,name=statement_handle,json=statementHandle,proto3" json:"statement_handle,omitempty"` +} + +func (x *TicketStatementQuery) Reset() { + *x = TicketStatementQuery{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TicketStatementQuery) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TicketStatementQuery) ProtoMessage() {} + +func (x *TicketStatementQuery) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[14] + 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 TicketStatementQuery.ProtoReflect.Descriptor instead. +func (*TicketStatementQuery) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{14} +} + +func (x *TicketStatementQuery) GetStatementHandle() []byte { + if x != nil { + return x.StatementHandle + } + return nil +} + +// +// Represents an instance of executing a prepared statement. Used in the command member of FlightDescriptor for +// the following RPC calls: +// - GetSchema: return the Arrow schema of the query. +// Fields on this schema may contain the following metadata: +// - ARROW:FLIGHT:SQL:CATALOG_NAME - Table's catalog name +// - ARROW:FLIGHT:SQL:DB_SCHEMA_NAME - Database schema name +// - ARROW:FLIGHT:SQL:TABLE_NAME - Table name +// - ARROW:FLIGHT:SQL:TYPE_NAME - The data source-specific name for the data type of the column. +// - ARROW:FLIGHT:SQL:PRECISION - Column precision/size +// - ARROW:FLIGHT:SQL:SCALE - Column scale/decimal digits if applicable +// - ARROW:FLIGHT:SQL:IS_AUTO_INCREMENT - "1" indicates if the column is auto incremented, "0" otherwise. +// - ARROW:FLIGHT:SQL:IS_CASE_SENSITIVE - "1" indicates if the column is case sensitive, "0" otherwise. +// - ARROW:FLIGHT:SQL:IS_READ_ONLY - "1" indicates if the column is read only, "0" otherwise. +// - ARROW:FLIGHT:SQL:IS_SEARCHABLE - "1" indicates if the column is searchable via WHERE clause, "0" otherwise. +// - DoPut: bind parameter values. All of the bound parameter sets will be executed as a single atomic execution. +// - GetFlightInfo: execute the prepared statement instance. +type CommandPreparedStatementQuery struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Opaque handle for the prepared statement on the server. + PreparedStatementHandle []byte `protobuf:"bytes,1,opt,name=prepared_statement_handle,json=preparedStatementHandle,proto3" json:"prepared_statement_handle,omitempty"` +} + +func (x *CommandPreparedStatementQuery) Reset() { + *x = CommandPreparedStatementQuery{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandPreparedStatementQuery) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandPreparedStatementQuery) ProtoMessage() {} + +func (x *CommandPreparedStatementQuery) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[15] + 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 CommandPreparedStatementQuery.ProtoReflect.Descriptor instead. +func (*CommandPreparedStatementQuery) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{15} +} + +func (x *CommandPreparedStatementQuery) GetPreparedStatementHandle() []byte { + if x != nil { + return x.PreparedStatementHandle + } + return nil +} + +// +// Represents a SQL update query. Used in the command member of FlightDescriptor +// for the the RPC call DoPut to cause the server to execute the included SQL update. +type CommandStatementUpdate struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The SQL syntax. + Query string `protobuf:"bytes,1,opt,name=query,proto3" json:"query,omitempty"` +} + +func (x *CommandStatementUpdate) Reset() { + *x = CommandStatementUpdate{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandStatementUpdate) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandStatementUpdate) ProtoMessage() {} + +func (x *CommandStatementUpdate) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[16] + 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 CommandStatementUpdate.ProtoReflect.Descriptor instead. +func (*CommandStatementUpdate) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{16} +} + +func (x *CommandStatementUpdate) GetQuery() string { + if x != nil { + return x.Query + } + return "" +} + +// +// Represents a SQL update query. Used in the command member of FlightDescriptor +// for the the RPC call DoPut to cause the server to execute the included +// prepared statement handle as an update. +type CommandPreparedStatementUpdate struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Opaque handle for the prepared statement on the server. + PreparedStatementHandle []byte `protobuf:"bytes,1,opt,name=prepared_statement_handle,json=preparedStatementHandle,proto3" json:"prepared_statement_handle,omitempty"` +} + +func (x *CommandPreparedStatementUpdate) Reset() { + *x = CommandPreparedStatementUpdate{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CommandPreparedStatementUpdate) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommandPreparedStatementUpdate) ProtoMessage() {} + +func (x *CommandPreparedStatementUpdate) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[17] + 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 CommandPreparedStatementUpdate.ProtoReflect.Descriptor instead. +func (*CommandPreparedStatementUpdate) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{17} +} + +func (x *CommandPreparedStatementUpdate) GetPreparedStatementHandle() []byte { + if x != nil { + return x.PreparedStatementHandle + } + return nil +} + +// +// Returned from the RPC call DoPut when a CommandStatementUpdate +// CommandPreparedStatementUpdate was in the request, containing +// results from the update. +type DoPutUpdateResult struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The number of records updated. A return value of -1 represents + // an unknown updated record count. + RecordCount int64 `protobuf:"varint,1,opt,name=record_count,json=recordCount,proto3" json:"record_count,omitempty"` +} + +func (x *DoPutUpdateResult) Reset() { + *x = DoPutUpdateResult{} + if protoimpl.UnsafeEnabled { + mi := &file_FlightSql_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DoPutUpdateResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DoPutUpdateResult) ProtoMessage() {} + +func (x *DoPutUpdateResult) ProtoReflect() protoreflect.Message { + mi := &file_FlightSql_proto_msgTypes[18] + 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 DoPutUpdateResult.ProtoReflect.Descriptor instead. +func (*DoPutUpdateResult) Descriptor() ([]byte, []int) { + return file_FlightSql_proto_rawDescGZIP(), []int{18} +} + +func (x *DoPutUpdateResult) GetRecordCount() int64 { + if x != nil { + return x.RecordCount + } + return 0 +} + +var file_FlightSql_proto_extTypes = []protoimpl.ExtensionInfo{ + { + ExtendedType: (*descriptor.MessageOptions)(nil), + ExtensionType: (*bool)(nil), + Field: 1000, + Name: "arrow.flight.protocol.sql.experimental", + Tag: "varint,1000,opt,name=experimental", + Filename: "FlightSql.proto", + }, +} + +// Extension fields to descriptor.MessageOptions. +var ( + // optional bool experimental = 1000; + E_Experimental = &file_FlightSql_proto_extTypes[0] +) + +var File_FlightSql_proto protoreflect.FileDescriptor + +var file_FlightSql_proto_rawDesc = []byte{ + 0x0a, 0x0f, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x53, 0x71, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x12, 0x19, 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, 0x1a, 0x20, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x2c, + 0x0a, 0x11, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x71, 0x6c, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0d, 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x3a, 0x03, 0xc0, 0x3e, 0x01, 0x22, 0x4d, 0x0a, 0x16, + 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x58, 0x64, 0x62, 0x63, 0x54, 0x79, + 0x70, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x20, 0x0a, 0x09, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x08, 0x64, 0x61, 0x74, + 0x61, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01, 0x3a, 0x03, 0xc0, 0x3e, 0x01, 0x42, 0x0c, 0x0a, + 0x0a, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0x19, 0x0a, 0x12, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, + 0x73, 0x3a, 0x03, 0xc0, 0x3e, 0x01, 0x22, 0xa0, 0x01, 0x0a, 0x13, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x47, 0x65, 0x74, 0x44, 0x62, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x12, 0x1d, + 0x0a, 0x07, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, + 0x00, 0x52, 0x07, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x88, 0x01, 0x01, 0x12, 0x3c, 0x0a, + 0x18, 0x64, 0x62, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, + 0x01, 0x52, 0x15, 0x64, 0x62, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x46, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x88, 0x01, 0x01, 0x3a, 0x03, 0xc0, 0x3e, 0x01, + 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x42, 0x1b, 0x0a, 0x19, + 0x5f, 0x64, 0x62, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x22, 0xc3, 0x02, 0x0a, 0x10, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x1d, + 0x0a, 0x07, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, + 0x00, 0x52, 0x07, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x88, 0x01, 0x01, 0x12, 0x3c, 0x0a, + 0x18, 0x64, 0x62, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, + 0x01, 0x52, 0x15, 0x64, 0x62, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x46, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x3e, 0x0a, 0x19, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, + 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, + 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x1f, 0x0a, 0x0b, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, + 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x3a, 0x03, 0xc0, 0x3e, 0x01, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x63, 0x61, 0x74, + 0x61, 0x6c, 0x6f, 0x67, 0x42, 0x1b, 0x0a, 0x19, 0x5f, 0x64, 0x62, 0x5f, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, + 0x6e, 0x42, 0x1c, 0x0a, 0x1a, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x22, + 0x1b, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x54, 0x79, 0x70, 0x65, 0x73, 0x3a, 0x03, 0xc0, 0x3e, 0x01, 0x22, 0x8d, 0x01, 0x0a, + 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x50, 0x72, 0x69, 0x6d, 0x61, + 0x72, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x1d, 0x0a, 0x07, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, + 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x07, 0x63, 0x61, 0x74, 0x61, 0x6c, + 0x6f, 0x67, 0x88, 0x01, 0x01, 0x12, 0x20, 0x0a, 0x09, 0x64, 0x62, 0x5f, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x08, 0x64, 0x62, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x3a, 0x03, 0xc0, + 0x3e, 0x01, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x42, 0x0c, + 0x0a, 0x0a, 0x5f, 0x64, 0x62, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x8e, 0x01, 0x0a, + 0x16, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x45, 0x78, 0x70, 0x6f, 0x72, + 0x74, 0x65, 0x64, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x1d, 0x0a, 0x07, 0x63, 0x61, 0x74, 0x61, 0x6c, + 0x6f, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x07, 0x63, 0x61, 0x74, 0x61, + 0x6c, 0x6f, 0x67, 0x88, 0x01, 0x01, 0x12, 0x20, 0x0a, 0x09, 0x64, 0x62, 0x5f, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x08, 0x64, 0x62, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x3a, 0x03, + 0xc0, 0x3e, 0x01, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x42, + 0x0c, 0x0a, 0x0a, 0x5f, 0x64, 0x62, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x8e, 0x01, + 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, + 0x72, 0x74, 0x65, 0x64, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x1d, 0x0a, 0x07, 0x63, 0x61, 0x74, 0x61, + 0x6c, 0x6f, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x07, 0x63, 0x61, 0x74, + 0x61, 0x6c, 0x6f, 0x67, 0x88, 0x01, 0x01, 0x12, 0x20, 0x0a, 0x09, 0x64, 0x62, 0x5f, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x08, 0x64, 0x62, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x3a, + 0x03, 0xc0, 0x3e, 0x01, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, + 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x64, 0x62, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xab, + 0x02, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x43, 0x72, 0x6f, + 0x73, 0x73, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x22, 0x0a, 0x0a, 0x70, + 0x6b, 0x5f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, + 0x00, 0x52, 0x09, 0x70, 0x6b, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x88, 0x01, 0x01, 0x12, + 0x25, 0x0a, 0x0c, 0x70, 0x6b, 0x5f, 0x64, 0x62, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0a, 0x70, 0x6b, 0x44, 0x62, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x19, 0x0a, 0x08, 0x70, 0x6b, 0x5f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x6b, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x12, 0x22, 0x0a, 0x0a, 0x66, 0x6b, 0x5f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x09, 0x66, 0x6b, 0x43, 0x61, 0x74, 0x61, 0x6c, + 0x6f, 0x67, 0x88, 0x01, 0x01, 0x12, 0x25, 0x0a, 0x0c, 0x66, 0x6b, 0x5f, 0x64, 0x62, 0x5f, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x03, 0x52, 0x0a, 0x66, + 0x6b, 0x44, 0x62, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x19, 0x0a, 0x08, + 0x66, 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x66, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x3a, 0x03, 0xc0, 0x3e, 0x01, 0x42, 0x0d, 0x0a, 0x0b, + 0x5f, 0x70, 0x6b, 0x5f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x42, 0x0f, 0x0a, 0x0d, 0x5f, + 0x70, 0x6b, 0x5f, 0x64, 0x62, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x0d, 0x0a, 0x0b, + 0x5f, 0x66, 0x6b, 0x5f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x42, 0x0f, 0x0a, 0x0d, 0x5f, + 0x66, 0x6b, 0x5f, 0x64, 0x62, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x41, 0x0a, 0x24, + 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, 0x65, 0x70, + 0x61, 0x72, 0x65, 0x64, 0x53, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x3a, 0x03, 0xc0, 0x3e, 0x01, 0x22, + 0xb8, 0x01, 0x0a, 0x23, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x64, 0x53, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3a, 0x0a, 0x19, 0x70, 0x72, 0x65, 0x70, 0x61, + 0x72, 0x65, 0x64, 0x5f, 0x73, 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, 0x12, 0x25, 0x0a, 0x0e, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x5f, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x64, 0x61, 0x74, + 0x61, 0x73, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x61, + 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x3a, 0x03, 0xc0, 0x3e, 0x01, 0x22, 0x66, 0x0a, 0x23, 0x41, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, + 0x64, 0x53, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x3a, 0x0a, 0x19, 0x70, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x64, 0x5f, 0x73, 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, 0x32, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x3a, 0x03, 0xc0, 0x3e, 0x01, 0x22, 0x46, 0x0a, 0x14, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x29, + 0x0a, 0x10, 0x73, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x68, 0x61, 0x6e, 0x64, + 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x3a, 0x03, 0xc0, 0x3e, 0x01, 0x22, 0x60, + 0x0a, 0x1d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, + 0x64, 0x53, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, + 0x3a, 0x0a, 0x19, 0x70, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x64, 0x5f, 0x73, 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, 0x33, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x3a, 0x03, 0xc0, 0x3e, 0x01, 0x22, 0x61, 0x0a, 0x1e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x64, 0x53, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x3a, 0x0a, 0x19, 0x70, 0x72, 0x65, 0x70, 0x61, + 0x72, 0x65, 0x64, 0x5f, 0x73, 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, 0x2a, 0x8c, 0x16, 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, 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, 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, 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, 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, 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, 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, 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, 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, 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, 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, 0x5b, 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, 0x37, 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, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x66, 0x6c, + 0x69, 0x67, 0x68, 0x74, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_FlightSql_proto_rawDescOnce sync.Once + file_FlightSql_proto_rawDescData = file_FlightSql_proto_rawDesc +) + +func file_FlightSql_proto_rawDescGZIP() []byte { + file_FlightSql_proto_rawDescOnce.Do(func() { + file_FlightSql_proto_rawDescData = protoimpl.X.CompressGZIP(file_FlightSql_proto_rawDescData) + }) + return file_FlightSql_proto_rawDescData +} + +var file_FlightSql_proto_enumTypes = make([]protoimpl.EnumInfo, 21) +var file_FlightSql_proto_msgTypes = make([]protoimpl.MessageInfo, 19) +var file_FlightSql_proto_goTypes = []interface{}{ + (SqlInfo)(0), // 0: arrow.flight.protocol.sql.SqlInfo + (SqlSupportedCaseSensitivity)(0), // 1: arrow.flight.protocol.sql.SqlSupportedCaseSensitivity + (SqlNullOrdering)(0), // 2: arrow.flight.protocol.sql.SqlNullOrdering + (SupportedSqlGrammar)(0), // 3: arrow.flight.protocol.sql.SupportedSqlGrammar + (SupportedAnsi92SqlGrammarLevel)(0), // 4: arrow.flight.protocol.sql.SupportedAnsi92SqlGrammarLevel + (SqlOuterJoinsSupportLevel)(0), // 5: arrow.flight.protocol.sql.SqlOuterJoinsSupportLevel + (SqlSupportedGroupBy)(0), // 6: arrow.flight.protocol.sql.SqlSupportedGroupBy + (SqlSupportedElementActions)(0), // 7: arrow.flight.protocol.sql.SqlSupportedElementActions + (SqlSupportedPositionedCommands)(0), // 8: arrow.flight.protocol.sql.SqlSupportedPositionedCommands + (SqlSupportedSubqueries)(0), // 9: arrow.flight.protocol.sql.SqlSupportedSubqueries + (SqlSupportedUnions)(0), // 10: arrow.flight.protocol.sql.SqlSupportedUnions + (SqlTransactionIsolationLevel)(0), // 11: arrow.flight.protocol.sql.SqlTransactionIsolationLevel + (SqlSupportedTransactions)(0), // 12: arrow.flight.protocol.sql.SqlSupportedTransactions + (SqlSupportedResultSetType)(0), // 13: arrow.flight.protocol.sql.SqlSupportedResultSetType + (SqlSupportedResultSetConcurrency)(0), // 14: arrow.flight.protocol.sql.SqlSupportedResultSetConcurrency + (SqlSupportsConvert)(0), // 15: arrow.flight.protocol.sql.SqlSupportsConvert + (XdbcDataType)(0), // 16: arrow.flight.protocol.sql.XdbcDataType + (XdbcDatetimeSubcode)(0), // 17: arrow.flight.protocol.sql.XdbcDatetimeSubcode + (Nullable)(0), // 18: arrow.flight.protocol.sql.Nullable + (Searchable)(0), // 19: arrow.flight.protocol.sql.Searchable + (UpdateDeleteRules)(0), // 20: arrow.flight.protocol.sql.UpdateDeleteRules + (*CommandGetSqlInfo)(nil), // 21: arrow.flight.protocol.sql.CommandGetSqlInfo + (*CommandGetXdbcTypeInfo)(nil), // 22: arrow.flight.protocol.sql.CommandGetXdbcTypeInfo + (*CommandGetCatalogs)(nil), // 23: arrow.flight.protocol.sql.CommandGetCatalogs + (*CommandGetDbSchemas)(nil), // 24: arrow.flight.protocol.sql.CommandGetDbSchemas + (*CommandGetTables)(nil), // 25: arrow.flight.protocol.sql.CommandGetTables + (*CommandGetTableTypes)(nil), // 26: arrow.flight.protocol.sql.CommandGetTableTypes + (*CommandGetPrimaryKeys)(nil), // 27: arrow.flight.protocol.sql.CommandGetPrimaryKeys + (*CommandGetExportedKeys)(nil), // 28: arrow.flight.protocol.sql.CommandGetExportedKeys + (*CommandGetImportedKeys)(nil), // 29: arrow.flight.protocol.sql.CommandGetImportedKeys + (*CommandGetCrossReference)(nil), // 30: arrow.flight.protocol.sql.CommandGetCrossReference + (*ActionCreatePreparedStatementRequest)(nil), // 31: arrow.flight.protocol.sql.ActionCreatePreparedStatementRequest + (*ActionCreatePreparedStatementResult)(nil), // 32: arrow.flight.protocol.sql.ActionCreatePreparedStatementResult + (*ActionClosePreparedStatementRequest)(nil), // 33: arrow.flight.protocol.sql.ActionClosePreparedStatementRequest + (*CommandStatementQuery)(nil), // 34: arrow.flight.protocol.sql.CommandStatementQuery + (*TicketStatementQuery)(nil), // 35: arrow.flight.protocol.sql.TicketStatementQuery + (*CommandPreparedStatementQuery)(nil), // 36: arrow.flight.protocol.sql.CommandPreparedStatementQuery + (*CommandStatementUpdate)(nil), // 37: arrow.flight.protocol.sql.CommandStatementUpdate + (*CommandPreparedStatementUpdate)(nil), // 38: arrow.flight.protocol.sql.CommandPreparedStatementUpdate + (*DoPutUpdateResult)(nil), // 39: arrow.flight.protocol.sql.DoPutUpdateResult + (*descriptor.MessageOptions)(nil), // 40: google.protobuf.MessageOptions +} +var file_FlightSql_proto_depIdxs = []int32{ + 40, // 0: arrow.flight.protocol.sql.experimental:extendee -> google.protobuf.MessageOptions + 1, // [1:1] is the sub-list for method output_type + 1, // [1:1] is the sub-list for method input_type + 1, // [1:1] is the sub-list for extension type_name + 0, // [0:1] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_FlightSql_proto_init() } +func file_FlightSql_proto_init() { + if File_FlightSql_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_FlightSql_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandGetSqlInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandGetXdbcTypeInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandGetCatalogs); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandGetDbSchemas); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandGetTables); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandGetTableTypes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandGetPrimaryKeys); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandGetExportedKeys); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandGetImportedKeys); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandGetCrossReference); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ActionCreatePreparedStatementRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ActionCreatePreparedStatementResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ActionClosePreparedStatementRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandStatementQuery); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TicketStatementQuery); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandPreparedStatementQuery); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandStatementUpdate); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CommandPreparedStatementUpdate); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_FlightSql_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DoPutUpdateResult); 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{}{} + file_FlightSql_proto_msgTypes[4].OneofWrappers = []interface{}{} + file_FlightSql_proto_msgTypes[6].OneofWrappers = []interface{}{} + file_FlightSql_proto_msgTypes[7].OneofWrappers = []interface{}{} + file_FlightSql_proto_msgTypes[8].OneofWrappers = []interface{}{} + file_FlightSql_proto_msgTypes[9].OneofWrappers = []interface{}{} + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_FlightSql_proto_rawDesc, + NumEnums: 21, + NumMessages: 19, + NumExtensions: 1, + NumServices: 0, + }, + GoTypes: file_FlightSql_proto_goTypes, + DependencyIndexes: file_FlightSql_proto_depIdxs, + EnumInfos: file_FlightSql_proto_enumTypes, + MessageInfos: file_FlightSql_proto_msgTypes, + ExtensionInfos: file_FlightSql_proto_extTypes, + }.Build() + File_FlightSql_proto = out.File + file_FlightSql_proto_rawDesc = nil + file_FlightSql_proto_goTypes = nil + file_FlightSql_proto_depIdxs = nil +} diff --git a/go/arrow/flight/record_batch_reader.go b/go/arrow/flight/record_batch_reader.go index dd2c29f0bf5d5..75e09f2008f6b 100644 --- a/go/arrow/flight/record_batch_reader.go +++ b/go/arrow/flight/record_batch_reader.go @@ -22,6 +22,8 @@ import ( "sync/atomic" "github.com/apache/arrow/go/v10/arrow" + "github.com/apache/arrow/go/v10/arrow/array" + "github.com/apache/arrow/go/v10/arrow/arrio" "github.com/apache/arrow/go/v10/arrow/internal/debug" "github.com/apache/arrow/go/v10/arrow/ipc" "github.com/apache/arrow/go/v10/arrow/memory" @@ -36,6 +38,7 @@ type DataStreamReader interface { type dataMessageReader struct { rdr DataStreamReader + peeked *FlightData refCount int64 msg *ipc.Message @@ -44,7 +47,18 @@ type dataMessageReader struct { } func (d *dataMessageReader) Message() (*ipc.Message, error) { - fd, err := d.rdr.Recv() + var ( + fd *FlightData + err error + ) + + if d.peeked != nil { + fd = d.peeked + d.peeked = nil + } else { + fd, err = d.rdr.Recv() + } + if err != nil { if d.msg != nil { // clear the previous message in the error case @@ -117,12 +131,34 @@ func (r *Reader) LatestFlightDescriptor() *FlightDescriptor { return r.dmr.descr } +// Chunk is a convenience function to return a chunk of the flight stream +// returning the RecordBatch along with the FlightDescriptor and any AppMetadata. +// Each of these can be retrieved separately with their respective functions, +// this is just a convenience to retrieve all three with one function call. +func (r *Reader) Chunk() StreamChunk { + return StreamChunk{ + Data: r.Record(), + Desc: r.dmr.descr, + AppMetadata: r.dmr.lastAppMetadata, + } +} + // NewRecordReader constructs an ipc reader using the flight data stream reader // as the source of the ipc messages, opts passed will be passed to the underlying // ipc.Reader such as ipc.WithSchema and ipc.WithAllocator func NewRecordReader(r DataStreamReader, opts ...ipc.Option) (*Reader, error) { + // peek the first message for a descriptor + data, err := r.Recv() + if err != nil { + return nil, err + } + rdr := &Reader{dmr: &dataMessageReader{rdr: r}} - var err error + rdr.dmr.descr = data.FlightDescriptor + if len(data.DataHeader) > 0 { + rdr.dmr.peeked = data + } + if rdr.Reader, err = ipc.NewReaderFromMessageReader(rdr.dmr, opts...); err != nil { return nil, fmt.Errorf("arrow/flight: could not create flight reader: %w", err) } @@ -144,3 +180,46 @@ func DeserializeSchema(info []byte, mem memory.Allocator) (*arrow.Schema, error) defer rdr.Release() return rdr.Schema(), nil } + +// StreamChunk represents a single chunk of a FlightData stream +type StreamChunk struct { + Data arrow.Record + Desc *FlightDescriptor + AppMetadata []byte + Err error +} + +// MessageReader is an interface representing a RecordReader +// that also provides StreamChunks and/or the ability to retrieve +// FlightDescriptors and AppMetadata from the flight stream +type MessageReader interface { + array.RecordReader + arrio.Reader + Err() error + Chunk() StreamChunk + LatestFlightDescriptor() *FlightDescriptor + LatestAppMetadata() []byte +} + +// StreamChunksFromReader is a convenience function to populate a channel +// from a record reader. It is intended to be run using a separate goroutine +// by calling `go flight.StreamChunksFromReader(rdr, ch)`. +// +// If the record reader panics, an error chunk will get sent on the channel. +// +// This will close the channel and release the reader when it completes. +func StreamChunksFromReader(rdr array.RecordReader, ch chan<- StreamChunk) { + defer close(ch) + defer func() { + if err := recover(); err != nil { + ch <- StreamChunk{Err: fmt.Errorf("panic while reading: %s", err)} + } + }() + + defer rdr.Release() + for rdr.Next() { + rec := rdr.Record() + rec.Retain() + ch <- StreamChunk{Data: rec} + } +} diff --git a/go/arrow/flight/record_batch_writer.go b/go/arrow/flight/record_batch_writer.go index ba907db22019c..efa2773c4839a 100644 --- a/go/arrow/flight/record_batch_writer.go +++ b/go/arrow/flight/record_batch_writer.go @@ -60,6 +60,12 @@ type Writer struct { pw *flightPayloadWriter } +// WriteMetadata writes a payload message to the stream containing only +// the specified app metadata. +func (w *Writer) WriteMetadata(appMetadata []byte) error { + return w.pw.w.Send(&FlightData{AppMetadata: appMetadata}) +} + // SetFlightDescriptor sets the flight descriptor into the next payload that will // be written by the flight writer. It will only be put into the very next payload // and afterwards the writer will no longer keep it's pointer to the descriptor. @@ -107,3 +113,7 @@ func SerializeSchema(rec *arrow.Schema, mem memory.Allocator) []byte { w.Close() return buf.Bytes() } + +type MetadataWriter interface { + WriteMetadata([]byte) error +} diff --git a/go/arrow/internal/flight_integration/scenario.go b/go/arrow/internal/flight_integration/scenario.go index 7dfd2d58d615e..c89334002d1da 100644 --- a/go/arrow/internal/flight_integration/scenario.go +++ b/go/arrow/internal/flight_integration/scenario.go @@ -24,11 +24,15 @@ import ( "io" "net" "os" + "reflect" "strconv" + "strings" "github.com/apache/arrow/go/v10/arrow" "github.com/apache/arrow/go/v10/arrow/array" "github.com/apache/arrow/go/v10/arrow/flight" + "github.com/apache/arrow/go/v10/arrow/flight/flightsql" + "github.com/apache/arrow/go/v10/arrow/flight/flightsql/schema_ref" "github.com/apache/arrow/go/v10/arrow/internal/arrjson" "github.com/apache/arrow/go/v10/arrow/internal/testing/types" "github.com/apache/arrow/go/v10/arrow/ipc" @@ -51,6 +55,8 @@ func GetScenario(name string, args ...string) Scenario { return &authBasicProtoTester{} case "middleware": return &middlewareScenarioTester{} + case "flight_sql": + return &flightSqlScenarioTester{} case "": if len(args) > 0 { return &defaultIntegrationTester{path: args[0]} @@ -517,3 +523,515 @@ func (m *middlewareScenarioTester) GetFlightInfo(ctx context.Context, desc *flig TotalBytes: -1, }, nil } + +var ( + // Schema to be returned for mocking the statement/prepared statement + // results. Must be the same across all languages + QuerySchema = arrow.NewSchema([]arrow.Field{{ + Name: "id", Type: arrow.PrimitiveTypes.Int64, Nullable: true, + Metadata: flightsql.NewColumnMetadataBuilder(). + TableName("test").IsAutoIncrement(true).IsCaseSensitive(false). + TypeName("type_test").SchemaName("schema_test").IsSearchable(true). + CatalogName("catalog_test").Precision(100).Metadata(), + }}, nil) +) + +const ( + updateStatementExpectedRows int64 = 10000 + updatePreparedStatementExpectedRows int64 = 20000 +) + +type flightSqlScenarioTester struct { + flightsql.BaseServer +} + +func (m *flightSqlScenarioTester) flightInfoForCommand(desc *flight.FlightDescriptor, schema *arrow.Schema) *flight.FlightInfo { + return &flight.FlightInfo{ + Endpoint: []*flight.FlightEndpoint{ + {Ticket: &flight.Ticket{Ticket: desc.Cmd}}, + }, + Schema: flight.SerializeSchema(schema, memory.DefaultAllocator), + FlightDescriptor: desc, + TotalRecords: -1, + TotalBytes: -1, + } +} + +func (m *flightSqlScenarioTester) MakeServer(port int) flight.Server { + srv := flight.NewServerWithMiddleware(nil) + srv.RegisterFlightService(flightsql.NewFlightServer(m)) + initServer(port, srv) + return srv +} + +func assertEq(expected, actual interface{}) error { + v := reflect.Indirect(reflect.ValueOf(actual)) + if !reflect.DeepEqual(expected, v.Interface()) { + return fmt.Errorf("expected: '%s', got: '%s'", expected, actual) + } + return nil +} + +func (m *flightSqlScenarioTester) RunClient(addr string, opts ...grpc.DialOption) error { + client, err := flightsql.NewClient(addr, nil, nil, opts...) + if err != nil { + return err + } + defer client.Close() + + if err := m.ValidateMetadataRetrieval(client); err != nil { + return err + } + + if err := m.ValidateStatementExecution(client); err != nil { + return err + } + + return m.ValidatePreparedStatementExecution(client) +} + +func (m *flightSqlScenarioTester) validate(expected *arrow.Schema, result *flight.FlightInfo, client *flightsql.Client) error { + rdr, err := client.DoGet(context.Background(), result.Endpoint[0].Ticket) + if err != nil { + return err + } + + if !expected.Equal(rdr.Schema()) { + return fmt.Errorf("expected: %s, got: %s", expected, rdr.Schema()) + } + return nil +} + +func (m *flightSqlScenarioTester) ValidateMetadataRetrieval(client *flightsql.Client) error { + var ( + catalog = "catalog" + dbSchemaFilterPattern = "db_schema_filter_pattern" + tableFilterPattern = "table_filter_pattern" + table = "table" + dbSchema = "db_schema" + tableTypes = []string{"table", "view"} + + ref = flightsql.TableRef{Catalog: &catalog, DBSchema: &dbSchema, Table: table} + pkRef = flightsql.TableRef{Catalog: proto.String("pk_catalog"), DBSchema: proto.String("pk_db_schema"), Table: "pk_table"} + fkRef = flightsql.TableRef{Catalog: proto.String("fk_catalog"), DBSchema: proto.String("fk_db_schema"), Table: "fk_table"} + + ctx = context.Background() + ) + + info, err := client.GetCatalogs(ctx) + if err != nil { + return err + } + if err := m.validate(schema_ref.Catalogs, info, client); err != nil { + return err + } + + info, err = client.GetDBSchemas(ctx, &flightsql.GetDBSchemasOpts{Catalog: &catalog, DbSchemaFilterPattern: &dbSchemaFilterPattern}) + if err != nil { + return err + } + if err = m.validate(schema_ref.DBSchemas, info, client); err != nil { + return err + } + + info, err = client.GetTables(ctx, &flightsql.GetTablesOpts{Catalog: &catalog, DbSchemaFilterPattern: &dbSchemaFilterPattern, TableNameFilterPattern: &tableFilterPattern, IncludeSchema: true, TableTypes: tableTypes}) + if err != nil { + return err + } + if err = m.validate(schema_ref.TablesWithIncludedSchema, info, client); err != nil { + return err + } + + info, err = client.GetTableTypes(ctx) + if err != nil { + return err + } + if err = m.validate(schema_ref.TableTypes, info, client); err != nil { + return err + } + + info, err = client.GetPrimaryKeys(ctx, ref) + if err != nil { + return err + } + if err = m.validate(schema_ref.PrimaryKeys, info, client); err != nil { + return err + } + + info, err = client.GetExportedKeys(ctx, ref) + if err != nil { + return err + } + if err = m.validate(schema_ref.ExportedKeys, info, client); err != nil { + return err + } + + info, err = client.GetImportedKeys(ctx, ref) + if err != nil { + return err + } + if err = m.validate(schema_ref.ImportedKeys, info, client); err != nil { + return err + } + + info, err = client.GetCrossReference(ctx, pkRef, fkRef) + if err != nil { + return err + } + if err = m.validate(schema_ref.CrossReference, info, client); err != nil { + return err + } + + info, err = client.GetXdbcTypeInfo(ctx, nil) + if err != nil { + return err + } + if err = m.validate(schema_ref.XdbcTypeInfo, info, client); err != nil { + return err + } + + info, err = client.GetSqlInfo(ctx, []flightsql.SqlInfo{flightsql.SqlInfoFlightSqlServerName, flightsql.SqlInfoFlightSqlServerReadOnly}) + if err != nil { + return err + } + if err = m.validate(schema_ref.SqlInfo, info, client); err != nil { + return err + } + + return nil +} + +func (m *flightSqlScenarioTester) ValidateStatementExecution(client *flightsql.Client) error { + ctx := context.Background() + info, err := client.Execute(ctx, "SELECT STATEMENT") + if err != nil { + return err + } + if err = m.validate(QuerySchema, info, client); err != nil { + return err + } + + updateResult, err := client.ExecuteUpdate(ctx, "UPDATE STATEMENT") + if err != nil { + return err + } + if updateResult != updateStatementExpectedRows { + return fmt.Errorf("expected 'UPDATE STATEMENT' return %d got %d", updateStatementExpectedRows, updateResult) + } + return nil +} + +func (m *flightSqlScenarioTester) ValidatePreparedStatementExecution(client *flightsql.Client) error { + ctx := context.Background() + prepared, err := client.Prepare(ctx, memory.DefaultAllocator, "SELECT PREPARED STATEMENT") + if err != nil { + return err + } + + arr, _, _ := array.FromJSON(memory.DefaultAllocator, arrow.PrimitiveTypes.Int64, strings.NewReader("[1]")) + defer arr.Release() + params := array.NewRecord(QuerySchema, []arrow.Array{arr}, 1) + prepared.SetParameters(params) + + info, err := prepared.Execute(ctx) + if err != nil { + return err + } + if err = m.validate(QuerySchema, info, client); err != nil { + return err + } + + if err = prepared.Close(ctx); err != nil { + return err + } + + updatePrepared, err := client.Prepare(ctx, memory.DefaultAllocator, "UPDATE PREPARED STATEMENT") + if err != nil { + return err + } + updateResult, err := updatePrepared.ExecuteUpdate(ctx) + if err != nil { + return err + } + + if updateResult != updatePreparedStatementExpectedRows { + return fmt.Errorf("expected 'UPDATE STATEMENT' return %d got %d", updatePreparedStatementExpectedRows, updateResult) + } + return updatePrepared.Close(ctx) +} + +func (m *flightSqlScenarioTester) doGetForTestCase(schema *arrow.Schema) chan flight.StreamChunk { + ch := make(chan flight.StreamChunk) + go func() { + ch <- flight.StreamChunk{Data: array.NewRecord(schema, []arrow.Array{}, 0)} + }() + return ch +} + +func (m *flightSqlScenarioTester) GetFlightInfoStatement(ctx context.Context, cmd flightsql.StatementQuery, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + if err := assertEq("SELECT STATEMENT", cmd.GetQuery()); err != nil { + return nil, err + } + + handle, err := flightsql.CreateStatementQueryTicket([]byte("SELECT STATEMENT HANDLE")) + if err != nil { + return nil, err + } + + return &flight.FlightInfo{ + Endpoint: []*flight.FlightEndpoint{ + {Ticket: &flight.Ticket{Ticket: handle}}, + }, + Schema: flight.SerializeSchema(QuerySchema, memory.DefaultAllocator), + FlightDescriptor: desc, + TotalRecords: -1, + TotalBytes: -1, + }, nil +} + +func (m *flightSqlScenarioTester) DoGetStatement(ctx context.Context, cmd flightsql.StatementQueryTicket) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return QuerySchema, m.doGetForTestCase(QuerySchema), nil +} + +func (m *flightSqlScenarioTester) GetFlightInfoPreparedStatement(_ context.Context, cmd flightsql.PreparedStatementQuery, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + err := assertEq([]byte("SELECT PREPARED STATEMENT HANDLE"), cmd.GetPreparedStatementHandle()) + if err != nil { + return nil, err + } + return m.flightInfoForCommand(desc, QuerySchema), nil +} + +func (m *flightSqlScenarioTester) DoGetPreparedStatement(_ context.Context, cmd flightsql.PreparedStatementQuery) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return QuerySchema, m.doGetForTestCase(QuerySchema), nil +} + +func (m *flightSqlScenarioTester) GetFlightInfoCatalogs(_ context.Context, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return m.flightInfoForCommand(desc, schema_ref.Catalogs), nil +} + +func (m *flightSqlScenarioTester) DoGetCatalogs(_ context.Context) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return schema_ref.Catalogs, m.doGetForTestCase(schema_ref.Catalogs), nil +} + +func (m *flightSqlScenarioTester) GetFlightInfoXdbcTypeInfo(_ context.Context, cmd flightsql.GetXdbcTypeInfo, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return m.flightInfoForCommand(desc, schema_ref.XdbcTypeInfo), nil +} + +func (m *flightSqlScenarioTester) DoGetXdbcTypeInfo(context.Context, flightsql.GetXdbcTypeInfo) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return schema_ref.XdbcTypeInfo, m.doGetForTestCase(schema_ref.XdbcTypeInfo), nil +} + +func (m *flightSqlScenarioTester) GetFlightInfoSqlInfo(_ context.Context, cmd flightsql.GetSqlInfo, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + if err := assertEq(int(2), len(cmd.GetInfo())); err != nil { + return nil, err + } + if err := assertEq(flightsql.SqlInfoFlightSqlServerName, flightsql.SqlInfo(cmd.GetInfo()[0])); err != nil { + return nil, err + } + if err := assertEq(flightsql.SqlInfoFlightSqlServerReadOnly, flightsql.SqlInfo(cmd.GetInfo()[1])); err != nil { + return nil, err + } + + return m.flightInfoForCommand(desc, schema_ref.SqlInfo), nil +} + +func (m *flightSqlScenarioTester) DoGetSqlInfo(context.Context, flightsql.GetSqlInfo) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return schema_ref.SqlInfo, m.doGetForTestCase(schema_ref.SqlInfo), nil +} + +func (m *flightSqlScenarioTester) GetFlightInfoSchemas(_ context.Context, cmd flightsql.GetDBSchemas, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + if err := assertEq("catalog", cmd.GetCatalog()); err != nil { + return nil, err + } + + if err := assertEq("db_schema_filter_pattern", cmd.GetDBSchemaFilterPattern()); err != nil { + return nil, err + } + + return m.flightInfoForCommand(desc, schema_ref.DBSchemas), nil +} + +func (m *flightSqlScenarioTester) DoGetDBSchemas(context.Context, flightsql.GetDBSchemas) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return schema_ref.DBSchemas, m.doGetForTestCase(schema_ref.DBSchemas), nil +} + +func (m *flightSqlScenarioTester) GetFlightInfoTables(_ context.Context, cmd flightsql.GetTables, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + if err := assertEq("catalog", cmd.GetCatalog()); err != nil { + return nil, err + } + + if err := assertEq("db_schema_filter_pattern", cmd.GetDBSchemaFilterPattern()); err != nil { + return nil, err + } + + if err := assertEq("table_filter_pattern", cmd.GetTableNameFilterPattern()); err != nil { + return nil, err + } + + if err := assertEq(int(2), len(cmd.GetTableTypes())); err != nil { + return nil, err + } + + if err := assertEq("table", cmd.GetTableTypes()[0]); err != nil { + return nil, err + } + + if err := assertEq("view", cmd.GetTableTypes()[1]); err != nil { + return nil, err + } + + if err := assertEq(true, cmd.GetIncludeSchema()); err != nil { + return nil, err + } + + return m.flightInfoForCommand(desc, schema_ref.TablesWithIncludedSchema), nil +} + +func (m *flightSqlScenarioTester) DoGetTables(context.Context, flightsql.GetTables) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return schema_ref.TablesWithIncludedSchema, m.doGetForTestCase(schema_ref.TablesWithIncludedSchema), nil +} + +func (m *flightSqlScenarioTester) GetFlightInfoTableTypes(_ context.Context, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + return m.flightInfoForCommand(desc, schema_ref.TableTypes), nil +} + +func (m *flightSqlScenarioTester) DoGetTableTypes(context.Context) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return schema_ref.TableTypes, m.doGetForTestCase(schema_ref.TableTypes), nil +} + +func (m *flightSqlScenarioTester) GetFlightInfoPrimaryKeys(_ context.Context, cmd flightsql.TableRef, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + if err := assertEq("catalog", cmd.Catalog); err != nil { + return nil, err + } + + if err := assertEq("db_schema", cmd.DBSchema); err != nil { + return nil, err + } + + if err := assertEq("table", cmd.Table); err != nil { + return nil, err + } + + return m.flightInfoForCommand(desc, schema_ref.PrimaryKeys), nil +} + +func (m *flightSqlScenarioTester) DoGetPrimaryKeys(context.Context, flightsql.TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return schema_ref.PrimaryKeys, m.doGetForTestCase(schema_ref.PrimaryKeys), nil +} + +func (m *flightSqlScenarioTester) GetFlightInfoExportedKeys(_ context.Context, cmd flightsql.TableRef, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + if err := assertEq("catalog", cmd.Catalog); err != nil { + return nil, err + } + + if err := assertEq("db_schema", cmd.DBSchema); err != nil { + return nil, err + } + + if err := assertEq("table", cmd.Table); err != nil { + return nil, err + } + + return m.flightInfoForCommand(desc, schema_ref.ExportedKeys), nil +} + +func (m *flightSqlScenarioTester) DoGetExportedKeys(context.Context, flightsql.TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return schema_ref.ExportedKeys, m.doGetForTestCase(schema_ref.ExportedKeys), nil +} + +func (m *flightSqlScenarioTester) GetFlightInfoImportedKeys(_ context.Context, cmd flightsql.TableRef, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + if err := assertEq("catalog", cmd.Catalog); err != nil { + return nil, err + } + + if err := assertEq("db_schema", cmd.DBSchema); err != nil { + return nil, err + } + + if err := assertEq("table", cmd.Table); err != nil { + return nil, err + } + + return m.flightInfoForCommand(desc, schema_ref.ImportedKeys), nil +} + +func (m *flightSqlScenarioTester) DoGetImportedKeys(context.Context, flightsql.TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return schema_ref.ImportedKeys, m.doGetForTestCase(schema_ref.ImportedKeys), nil +} + +func (m *flightSqlScenarioTester) GetFlightInfoCrossReference(_ context.Context, cmd flightsql.CrossTableRef, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + if err := assertEq("pk_catalog", cmd.PKRef.Catalog); err != nil { + return nil, err + } + + if err := assertEq("pk_db_schema", cmd.PKRef.DBSchema); err != nil { + return nil, err + } + + if err := assertEq("pk_table", cmd.PKRef.Table); err != nil { + return nil, err + } + + if err := assertEq("fk_catalog", cmd.FKRef.Catalog); err != nil { + return nil, err + } + + if err := assertEq("fk_db_schema", cmd.FKRef.DBSchema); err != nil { + return nil, err + } + + if err := assertEq("fk_table", cmd.FKRef.Table); err != nil { + return nil, err + } + + return m.flightInfoForCommand(desc, schema_ref.TableTypes), nil +} + +func (m *flightSqlScenarioTester) DoGetCrossReference(context.Context, flightsql.CrossTableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) { + return schema_ref.CrossReference, m.doGetForTestCase(schema_ref.CrossReference), nil +} + +func (m *flightSqlScenarioTester) DoPutCommandStatementUpdate(_ context.Context, cmd flightsql.StatementUpdate) (int64, error) { + if err := assertEq("UPDATE STATEMENT", cmd.GetQuery()); err != nil { + return 0, err + } + + return updateStatementExpectedRows, nil +} + +func (m *flightSqlScenarioTester) CreatePreparedStatement(_ context.Context, request flightsql.ActionCreatePreparedStatementRequest) (res flightsql.ActionCreatePreparedStatementResult, err error) { + err = assertEq(true, request.GetQuery() == "SELECT PREPARED STATEMENT" || request.GetQuery() == "UPDATE PREPARED STATEMENT") + if err != nil { + return + } + + res.Handle = []byte(request.GetQuery() + " HANDLE") + return +} + +func (m *flightSqlScenarioTester) ClosePreparedStatement(context.Context, flightsql.ActionClosePreparedStatementRequest) error { + return nil +} + +func (m *flightSqlScenarioTester) DoPutPreparedStatementQuery(_ context.Context, cmd flightsql.PreparedStatementQuery, rdr flight.MessageReader, _ flight.MetadataWriter) error { + err := assertEq([]byte("SELECT PREPARED STATEMENT HANDLE"), cmd.GetPreparedStatementHandle()) + if err != nil { + return err + } + + actualSchema := rdr.Schema() + if err = assertEq(true, actualSchema.Equal(QuerySchema)); err != nil { + return err + } + + return nil +} + +func (m *flightSqlScenarioTester) DoPutPreparedStatementUpdate(_ context.Context, cmd flightsql.PreparedStatementUpdate, _ flight.MessageReader) (int64, error) { + err := assertEq([]byte("UPDATE PREPARED STATEMENT HANDLE"), cmd.GetPreparedStatementHandle()) + if err != nil { + return 0, err + } + + return updatePreparedStatementExpectedRows, nil +} diff --git a/go/arrow/ipc/cmd/arrow-cat/main.go b/go/arrow/ipc/cmd/arrow-cat/main.go index aa3818be658a6..0df83a30de42c 100644 --- a/go/arrow/ipc/cmd/arrow-cat/main.go +++ b/go/arrow/ipc/cmd/arrow-cat/main.go @@ -91,7 +91,7 @@ func processStream(w io.Writer, rin io.Reader) error { r, err := ipc.NewReader(rin, ipc.WithAllocator(mem)) if err != nil { if errors.Is(err, io.EOF) { - return nil + break } return err } diff --git a/go/arrow/ipc/ipc.go b/go/arrow/ipc/ipc.go index de1010b8d114d..71d810b1361c6 100644 --- a/go/arrow/ipc/ipc.go +++ b/go/arrow/ipc/ipc.go @@ -69,6 +69,7 @@ type config struct { codec flatbuf.CompressionType compressNP int ensureNativeEndian bool + noAutoSchema bool } func newConfig(opts ...Option) *config { @@ -150,6 +151,15 @@ func WithEnsureNativeEndian(v bool) Option { } } +// WithDelayedReadSchema alters the ipc.Reader behavior to delay attempting +// to read the schema from the stream until the first call to Next instead +// of immediately attempting to read a schema from the stream when created. +func WithDelayReadSchema(v bool) Option { + return func(cfg *config) { + cfg.noAutoSchema = v + } +} + var ( _ arrio.Reader = (*Reader)(nil) _ arrio.Writer = (*Writer)(nil) diff --git a/go/arrow/ipc/reader.go b/go/arrow/ipc/reader.go index 101e84b444bce..0af22bd95ca8f 100644 --- a/go/arrow/ipc/reader.go +++ b/go/arrow/ipc/reader.go @@ -44,10 +44,12 @@ type Reader struct { err error // types dictTypeMap - memo dictutils.Memo - readInitialDicts bool - done bool - swapEndianness bool + memo dictutils.Memo + readInitialDicts bool + done bool + swapEndianness bool + ensureNativeEndian bool + expectedSchema *arrow.Schema mem memory.Allocator } @@ -70,18 +72,16 @@ func NewReaderFromMessageReader(r MessageReader, opts ...Option) (reader *Reader r: r, refCount: 1, // types: make(dictTypeMap), - memo: dictutils.NewMemo(), - mem: cfg.alloc, + memo: dictutils.NewMemo(), + mem: cfg.alloc, + ensureNativeEndian: cfg.ensureNativeEndian, + expectedSchema: cfg.schema, } - err = rr.readSchema(cfg.schema) - if err != nil { - return nil, fmt.Errorf("arrow/ipc: could not read schema from stream: %w", err) - } - - if cfg.ensureNativeEndian && !rr.schema.IsNativeEndian() { - rr.swapEndianness = true - rr.schema = rr.schema.WithEndianness(endian.NativeEndian) + if !cfg.noAutoSchema { + if err := rr.readSchema(cfg.schema); err != nil { + return nil, err + } } return rr, nil @@ -96,7 +96,15 @@ func NewReader(r io.Reader, opts ...Option) (*Reader, error) { // underlying stream. func (r *Reader) Err() error { return r.err } -func (r *Reader) Schema() *arrow.Schema { return r.schema } +func (r *Reader) Schema() *arrow.Schema { + if r.schema == nil { + if err := r.readSchema(r.expectedSchema); err != nil { + r.err = fmt.Errorf("arrow/ipc: could not read schema from stream: %w", err) + r.done = true + } + } + return r.schema +} func (r *Reader) readSchema(schema *arrow.Schema) error { msg, err := r.r.Message() @@ -122,6 +130,11 @@ func (r *Reader) readSchema(schema *arrow.Schema) error { return errInconsistentSchema } + if r.ensureNativeEndian && !r.schema.IsNativeEndian() { + r.swapEndianness = true + r.schema = r.schema.WithEndianness(endian.NativeEndian) + } + return nil } @@ -202,6 +215,13 @@ func (r *Reader) next() bool { r.err = fmt.Errorf("arrow/ipc: unknown error while reading: %v", pErr) } }() + if r.schema == nil { + if err := r.readSchema(r.expectedSchema); err != nil { + r.err = fmt.Errorf("arrow/ipc: could not read schema from stream: %w", err) + r.done = true + return false + } + } if !r.readInitialDicts && !r.getInitialDicts() { return false diff --git a/go/arrow/math/float64_amd64.go b/go/arrow/math/float64_amd64.go index 87227d5fa822f..ef40cf613a3c9 100644 --- a/go/arrow/math/float64_amd64.go +++ b/go/arrow/math/float64_amd64.go @@ -16,6 +16,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build !noasm // +build !noasm package math diff --git a/go/arrow/math/float64_avx2_amd64.go b/go/arrow/math/float64_avx2_amd64.go index a6cdd04be7c5a..fd896abc79aab 100644 --- a/go/arrow/math/float64_avx2_amd64.go +++ b/go/arrow/math/float64_avx2_amd64.go @@ -16,6 +16,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build !noasm // +build !noasm package math diff --git a/go/arrow/math/float64_sse4_amd64.go b/go/arrow/math/float64_sse4_amd64.go index 469248ce1575d..d388cce72769a 100644 --- a/go/arrow/math/float64_sse4_amd64.go +++ b/go/arrow/math/float64_sse4_amd64.go @@ -16,6 +16,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build !noasm // +build !noasm package math diff --git a/go/arrow/math/int64_amd64.go b/go/arrow/math/int64_amd64.go index 2703bebd00a99..ed616e84264e8 100644 --- a/go/arrow/math/int64_amd64.go +++ b/go/arrow/math/int64_amd64.go @@ -16,6 +16,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build !noasm // +build !noasm package math diff --git a/go/arrow/math/int64_avx2_amd64.go b/go/arrow/math/int64_avx2_amd64.go index a0ddaa7658a9e..60e4c98ef2a89 100644 --- a/go/arrow/math/int64_avx2_amd64.go +++ b/go/arrow/math/int64_avx2_amd64.go @@ -16,6 +16,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build !noasm // +build !noasm package math diff --git a/go/arrow/math/int64_sse4_amd64.go b/go/arrow/math/int64_sse4_amd64.go index 16019080d6391..75f94f56b739a 100644 --- a/go/arrow/math/int64_sse4_amd64.go +++ b/go/arrow/math/int64_sse4_amd64.go @@ -16,6 +16,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build !noasm // +build !noasm package math diff --git a/go/arrow/math/uint64_amd64.go b/go/arrow/math/uint64_amd64.go index c6bc29c4e3e32..726d3425614f8 100644 --- a/go/arrow/math/uint64_amd64.go +++ b/go/arrow/math/uint64_amd64.go @@ -16,6 +16,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build !noasm // +build !noasm package math diff --git a/go/arrow/math/uint64_avx2_amd64.go b/go/arrow/math/uint64_avx2_amd64.go index 88b8bc55e14a9..7f953d8b8ef3a 100644 --- a/go/arrow/math/uint64_avx2_amd64.go +++ b/go/arrow/math/uint64_avx2_amd64.go @@ -16,6 +16,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build !noasm // +build !noasm package math diff --git a/go/arrow/math/uint64_sse4_amd64.go b/go/arrow/math/uint64_sse4_amd64.go index d8047f277fcc1..30f33135eb49c 100644 --- a/go/arrow/math/uint64_sse4_amd64.go +++ b/go/arrow/math/uint64_sse4_amd64.go @@ -16,6 +16,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:build !noasm // +build !noasm package math diff --git a/go/arrow/scalar/numeric.gen.go b/go/arrow/scalar/numeric.gen.go index 0dd0aca3d42c4..9d743f5e9de80 100644 --- a/go/arrow/scalar/numeric.gen.go +++ b/go/arrow/scalar/numeric.gen.go @@ -23,7 +23,7 @@ import ( "reflect" "unsafe" - "github.com/apache/arrow/go/v10/arrow" + "github.com/apache/arrow/go/v10/arrow" ) type Int8 struct { diff --git a/go/go.mod b/go/go.mod index 95a5538d0a5f9..18a1c9abfa773 100644 --- a/go/go.mod +++ b/go/go.mod @@ -24,6 +24,7 @@ require ( github.com/apache/thrift v0.16.0 github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815 github.com/goccy/go-json v0.9.10 + github.com/golang/protobuf v1.5.2 github.com/golang/snappy v0.0.4 github.com/google/flatbuffers v2.0.6+incompatible github.com/klauspost/asmfmt v1.3.2