diff --git a/cpp/src/arrow/flight/flight_internals_test.cc b/cpp/src/arrow/flight/flight_internals_test.cc index b32036a2c973f..5feb310fc14a2 100644 --- a/cpp/src/arrow/flight/flight_internals_test.cc +++ b/cpp/src/arrow/flight/flight_internals_test.cc @@ -191,33 +191,33 @@ TEST(FlightTypes, FlightEndpoint) { Timestamp expiration_time( std::chrono::duration_cast(expiration_time_duration)); std::vector values = { - {{""}, {}, std::nullopt}, - {{"foo"}, {}, std::nullopt}, - {{"bar"}, {}, std::nullopt}, - {{"foo"}, {}, expiration_time}, - {{"foo"}, {location1}, std::nullopt}, - {{"bar"}, {location1}, std::nullopt}, - {{"foo"}, {location2}, std::nullopt}, - {{"foo"}, {location1, location2}, std::nullopt}, + {{""}, {}, std::nullopt, {}}, + {{"foo"}, {}, std::nullopt, {}}, + {{"bar"}, {}, std::nullopt, {"\xDE\xAD\xBE\xEF"}}, + {{"foo"}, {}, expiration_time, {}}, + {{"foo"}, {location1}, std::nullopt, {}}, + {{"bar"}, {location1}, std::nullopt, {}}, + {{"foo"}, {location2}, std::nullopt, {}}, + {{"foo"}, {location1, location2}, std::nullopt, {"\xba\xdd\xca\xfe"}}, }; std::vector reprs = { " locations=[] " - "expiration_time=null>", + "expiration_time=null app_metadata=''>", " locations=[] " - "expiration_time=null>", + "expiration_time=null app_metadata=''>", " locations=[] " - "expiration_time=null>", + "expiration_time=null app_metadata='DEADBEEF'>", " locations=[] " - "expiration_time=2023-06-19 03:14:06.004339000>", + "expiration_time=2023-06-19 03:14:06.004339000 app_metadata=''>", " locations=" - "[grpc+tcp://localhost:1024] expiration_time=null>", + "[grpc+tcp://localhost:1024] expiration_time=null app_metadata=''>", " locations=" - "[grpc+tcp://localhost:1024] expiration_time=null>", + "[grpc+tcp://localhost:1024] expiration_time=null app_metadata=''>", " locations=" - "[grpc+tls://localhost:1024] expiration_time=null>", + "[grpc+tls://localhost:1024] expiration_time=null app_metadata=''>", " locations=" "[grpc+tcp://localhost:1024, grpc+tls://localhost:1024] " - "expiration_time=null>", + "expiration_time=null app_metadata='BADDCAFE'>", }; ASSERT_NO_FATAL_FAILURE(TestRoundtrip(values, reprs)); @@ -229,30 +229,35 @@ TEST(FlightTypes, FlightInfo) { Schema schema2({}); auto desc1 = FlightDescriptor::Command("foo"); auto desc2 = FlightDescriptor::Command("bar"); - auto endpoint1 = FlightEndpoint{Ticket{"foo"}, {}, std::nullopt}; - auto endpoint2 = FlightEndpoint{Ticket{"foo"}, {location}, std::nullopt}; + auto endpoint1 = FlightEndpoint{Ticket{"foo"}, {}, std::nullopt, ""}; + auto endpoint2 = + FlightEndpoint{Ticket{"foo"}, {location}, std::nullopt, "\xCA\xFE\xD0\x0D"}; std::vector values = { - MakeFlightInfo(schema1, desc1, {}, -1, -1, false), - MakeFlightInfo(schema1, desc2, {}, -1, -1, true), - MakeFlightInfo(schema2, desc1, {}, -1, -1, false), - MakeFlightInfo(schema1, desc1, {endpoint1}, -1, 42, true), - MakeFlightInfo(schema1, desc2, {endpoint1, endpoint2}, 64, -1, false), + MakeFlightInfo(schema1, desc1, {}, -1, -1, false, ""), + MakeFlightInfo(schema1, desc2, {}, -1, -1, true, ""), + MakeFlightInfo(schema2, desc1, {}, -1, -1, false, ""), + MakeFlightInfo(schema1, desc1, {endpoint1}, -1, 42, true, ""), + MakeFlightInfo(schema1, desc2, {endpoint1, endpoint2}, 64, -1, false, + "\xDE\xAD\xC0\xDE"), }; std::vector reprs = { " " - "endpoints=[] total_records=-1 total_bytes=-1 ordered=false>", + "endpoints=[] total_records=-1 total_bytes=-1 ordered=false app_metadata=''>", " " - "endpoints=[] total_records=-1 total_bytes=-1 ordered=true>", + "endpoints=[] total_records=-1 total_bytes=-1 ordered=true app_metadata=''>", " " - "endpoints=[] total_records=-1 total_bytes=-1 ordered=false>", + "endpoints=[] total_records=-1 total_bytes=-1 ordered=false app_metadata=''>", " " "endpoints=[ locations=[] " - "expiration_time=null>] total_records=-1 total_bytes=42 ordered=true>", + "expiration_time=null app_metadata=''>] total_records=-1 total_bytes=42 " + "ordered=true app_metadata=''>", " " "endpoints=[ locations=[] " - "expiration_time=null>, " - "locations=[grpc+tcp://localhost:1234] expiration_time=null>] " - "total_records=64 total_bytes=-1 ordered=false>", + "expiration_time=null app_metadata=''>, " + "locations=[grpc+tcp://localhost:1234] expiration_time=null " + "app_metadata='CAFED00D'>] " + "total_records=64 total_bytes=-1 ordered=false app_metadata='DEADC0DE'>", }; ASSERT_NO_FATAL_FAILURE(TestRoundtrip(values, reprs)); @@ -262,8 +267,8 @@ TEST(FlightTypes, PollInfo) { ASSERT_OK_AND_ASSIGN(auto location, Location::ForGrpcTcp("localhost", 1234)); Schema schema({field("ints", int64())}); auto desc = FlightDescriptor::Command("foo"); - auto endpoint = FlightEndpoint{Ticket{"foo"}, {}, std::nullopt}; - auto info = MakeFlightInfo(schema, desc, {endpoint}, -1, 42, true); + auto endpoint = FlightEndpoint{Ticket{"foo"}, {}, std::nullopt, ""}; + auto info = MakeFlightInfo(schema, desc, {endpoint}, -1, 42, true, ""); // 2023-06-19 03:14:06.004330100 // We must use microsecond resolution here for portability. // std::chrono::system_clock::time_point may not provide nanosecond diff --git a/cpp/src/arrow/flight/integration_tests/flight_integration_test.cc b/cpp/src/arrow/flight/integration_tests/flight_integration_test.cc index 4a49dea31b99d..67c7ee85f59d3 100644 --- a/cpp/src/arrow/flight/integration_tests/flight_integration_test.cc +++ b/cpp/src/arrow/flight/integration_tests/flight_integration_test.cc @@ -73,6 +73,10 @@ TEST(FlightIntegration, ExpirationTimeRenewFlightEndpoint) { TEST(FlightIntegration, PollFlightInfo) { ASSERT_OK(RunScenario("poll_flight_info")); } +TEST(FlightIntegration, AppMetadataFlightInfoEndpoint) { + ASSERT_OK(RunScenario("app_metadata_flight_info_endpoint")); +} + TEST(FlightIntegration, FlightSql) { ASSERT_OK(RunScenario("flight_sql")); } TEST(FlightIntegration, FlightSqlExtension) { diff --git a/cpp/src/arrow/flight/integration_tests/test_integration.cc b/cpp/src/arrow/flight/integration_tests/test_integration.cc index 03e352ffa6770..31bffd7704474 100644 --- a/cpp/src/arrow/flight/integration_tests/test_integration.cc +++ b/cpp/src/arrow/flight/integration_tests/test_integration.cc @@ -815,6 +815,64 @@ class PollFlightInfoScenario : public Scenario { } }; +/// \brief The server used for testing app_metadata in FlightInfo and FlightEndpoint +class AppMetadataFlightInfoEndpointServer : public FlightServerBase { + public: + AppMetadataFlightInfoEndpointServer() : FlightServerBase() {} + + Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, + std::unique_ptr* info) override { + if (request.type != FlightDescriptor::CMD) { + return Status::Invalid("request descriptor should be of type CMD"); + } + + auto schema = arrow::schema({arrow::field("number", arrow::uint32(), false)}); + std::vector endpoints = { + FlightEndpoint{{}, {}, std::nullopt, request.cmd}}; + ARROW_ASSIGN_OR_RAISE(auto result, FlightInfo::Make(*schema, request, endpoints, -1, + -1, false, request.cmd)); + *info = std::make_unique(std::move(result)); + return Status::OK(); + } +}; + +/// \brief The AppMetadataFlightInfoEndpoint scenario. +/// +/// This tests that the client can receive and use the `app_metadata` field in +/// the FlightInfo and FlightEndpoint messages. +/// +/// The server only implements GetFlightInfo and will return a FlightInfo with a non- +/// empty app_metadata value that should match the app_metadata field in the +/// included FlightEndpoint. The value should be the same as the cmd bytes passed +/// in the call to GetFlightInfo by the client. +class AppMetadataFlightInfoEndpointScenario : public Scenario { + Status MakeServer(std::unique_ptr* server, + FlightServerOptions* options) override { + *server = std::make_unique(); + return Status::OK(); + } + + Status MakeClient(FlightClientOptions* options) override { return Status::OK(); } + + Status RunClient(std::unique_ptr client) override { + ARROW_ASSIGN_OR_RAISE(auto info, + client->GetFlightInfo(FlightDescriptor::Command("foobar"))); + if (info->app_metadata() != "foobar") { + return Status::Invalid("app_metadata should have been 'foobar', got: ", + info->app_metadata()); + } + if (info->endpoints().size() != 1) { + return Status::Invalid("should have gotten exactly one FlightEndpoint back, got: ", + info->endpoints().size()); + } + if (info->endpoints()[0].app_metadata != "foobar") { + return Status::Invalid("FlightEndpoint app_metadata should be 'foobar', got: ", + info->endpoints()[0].app_metadata); + } + return Status::OK(); + } +}; + /// \brief Schema to be returned for mocking the statement/prepared statement results. /// /// Must be the same across all languages. @@ -1897,6 +1955,9 @@ Status GetScenario(const std::string& scenario_name, std::shared_ptr* } else if (scenario_name == "poll_flight_info") { *out = std::make_shared(); return Status::OK(); + } else if (scenario_name == "app_metadata_flight_info_endpoint") { + *out = std::make_shared(); + return Status::OK(); } else if (scenario_name == "flight_sql") { *out = std::make_shared(); return Status::OK(); diff --git a/cpp/src/arrow/flight/perf_server.cc b/cpp/src/arrow/flight/perf_server.cc index a768840c0295d..40f6cbcbf0d82 100644 --- a/cpp/src/arrow/flight/perf_server.cc +++ b/cpp/src/arrow/flight/perf_server.cc @@ -196,7 +196,7 @@ class FlightPerfServer : public FlightServerBase { perf_request.stream_count() * perf_request.records_per_stream(); *info = std::make_unique( - MakeFlightInfo(*perf_schema_, request, endpoints, total_records, -1, false)); + MakeFlightInfo(*perf_schema_, request, endpoints, total_records, -1, false, "")); return Status::OK(); } diff --git a/cpp/src/arrow/flight/serialization_internal.cc b/cpp/src/arrow/flight/serialization_internal.cc index f85b451dccec3..64a40564afd72 100644 --- a/cpp/src/arrow/flight/serialization_internal.cc +++ b/cpp/src/arrow/flight/serialization_internal.cc @@ -177,6 +177,7 @@ Status FromProto(const pb::FlightEndpoint& pb_endpoint, FlightEndpoint* endpoint RETURN_NOT_OK(FromProto(pb_endpoint.expiration_time(), &expiration_time)); endpoint->expiration_time = std::move(expiration_time); } + endpoint->app_metadata = pb_endpoint.app_metadata(); return Status::OK(); } @@ -190,6 +191,7 @@ Status ToProto(const FlightEndpoint& endpoint, pb::FlightEndpoint* pb_endpoint) RETURN_NOT_OK(ToProto(endpoint.expiration_time.value(), pb_endpoint->mutable_expiration_time())); } + pb_endpoint->set_app_metadata(endpoint.app_metadata); return Status::OK(); } @@ -255,6 +257,7 @@ arrow::Result FromProto(const pb::FlightInfo& pb_info) { info.total_records = pb_info.total_records(); info.total_bytes = pb_info.total_bytes(); info.ordered = pb_info.ordered(); + info.app_metadata = pb_info.app_metadata(); return FlightInfo(std::move(info)); } @@ -296,6 +299,7 @@ Status ToProto(const FlightInfo& info, pb::FlightInfo* pb_info) { pb_info->set_total_records(info.total_records()); pb_info->set_total_bytes(info.total_bytes()); pb_info->set_ordered(info.ordered()); + pb_info->set_app_metadata(info.app_metadata()); return Status::OK(); } diff --git a/cpp/src/arrow/flight/sql/server.cc b/cpp/src/arrow/flight/sql/server.cc index 7a1c394fb413b..a6d197d15b2c0 100644 --- a/cpp/src/arrow/flight/sql/server.cc +++ b/cpp/src/arrow/flight/sql/server.cc @@ -930,7 +930,7 @@ arrow::Result> FlightSqlServerBase::GetFlightInfoSql } std::vector endpoints{ - FlightEndpoint{{descriptor.cmd}, {}, std::nullopt}}; + FlightEndpoint{{descriptor.cmd}, {}, std::nullopt, {}}}; ARROW_ASSIGN_OR_RAISE( auto result, FlightInfo::Make(*SqlSchema::GetSqlInfoSchema(), descriptor, endpoints, -1, -1, false)) diff --git a/cpp/src/arrow/flight/test_util.cc b/cpp/src/arrow/flight/test_util.cc index fed2b59abf56d..bf2f4c2b4effc 100644 --- a/cpp/src/arrow/flight/test_util.cc +++ b/cpp/src/arrow/flight/test_util.cc @@ -531,9 +531,11 @@ std::unique_ptr ExampleTestServer() { FlightInfo MakeFlightInfo(const Schema& schema, const FlightDescriptor& descriptor, const std::vector& endpoints, - int64_t total_records, int64_t total_bytes, bool ordered) { - EXPECT_OK_AND_ASSIGN(auto info, FlightInfo::Make(schema, descriptor, endpoints, - total_records, total_bytes, ordered)); + int64_t total_records, int64_t total_bytes, bool ordered, + std::string app_metadata) { + EXPECT_OK_AND_ASSIGN(auto info, + FlightInfo::Make(schema, descriptor, endpoints, total_records, + total_bytes, ordered, std::move(app_metadata))); return info; } @@ -602,11 +604,11 @@ std::vector ExampleFlightInfo() { Location location4 = *Location::ForGrpcTcp("foo4.bar.com", 12345); Location location5 = *Location::ForGrpcTcp("foo5.bar.com", 12345); - FlightEndpoint endpoint1({{"ticket-ints-1"}, {location1}, std::nullopt}); - FlightEndpoint endpoint2({{"ticket-ints-2"}, {location2}, std::nullopt}); - FlightEndpoint endpoint3({{"ticket-cmd"}, {location3}, std::nullopt}); - FlightEndpoint endpoint4({{"ticket-dicts-1"}, {location4}, std::nullopt}); - FlightEndpoint endpoint5({{"ticket-floats-1"}, {location5}, std::nullopt}); + FlightEndpoint endpoint1({{"ticket-ints-1"}, {location1}, std::nullopt, {}}); + FlightEndpoint endpoint2({{"ticket-ints-2"}, {location2}, std::nullopt, {}}); + FlightEndpoint endpoint3({{"ticket-cmd"}, {location3}, std::nullopt, {}}); + FlightEndpoint endpoint4({{"ticket-dicts-1"}, {location4}, std::nullopt, {}}); + FlightEndpoint endpoint5({{"ticket-floats-1"}, {location5}, std::nullopt, {}}); FlightDescriptor descr1{FlightDescriptor::PATH, "", {"examples", "ints"}}; FlightDescriptor descr2{FlightDescriptor::CMD, "my_command", {}}; @@ -619,10 +621,10 @@ std::vector ExampleFlightInfo() { auto schema4 = ExampleFloatSchema(); return { - MakeFlightInfo(*schema1, descr1, {endpoint1, endpoint2}, 1000, 100000, false), - MakeFlightInfo(*schema2, descr2, {endpoint3}, 1000, 100000, false), - MakeFlightInfo(*schema3, descr3, {endpoint4}, -1, -1, false), - MakeFlightInfo(*schema4, descr4, {endpoint5}, 1000, 100000, false), + MakeFlightInfo(*schema1, descr1, {endpoint1, endpoint2}, 1000, 100000, false, ""), + MakeFlightInfo(*schema2, descr2, {endpoint3}, 1000, 100000, false, ""), + MakeFlightInfo(*schema3, descr3, {endpoint4}, -1, -1, false, ""), + MakeFlightInfo(*schema4, descr4, {endpoint5}, 1000, 100000, false, ""), }; } diff --git a/cpp/src/arrow/flight/test_util.h b/cpp/src/arrow/flight/test_util.h index f299d358c539e..c0b42d9b90c5a 100644 --- a/cpp/src/arrow/flight/test_util.h +++ b/cpp/src/arrow/flight/test_util.h @@ -192,7 +192,8 @@ std::vector ExampleActionTypes(); ARROW_FLIGHT_EXPORT FlightInfo MakeFlightInfo(const Schema& schema, const FlightDescriptor& descriptor, const std::vector& endpoints, - int64_t total_records, int64_t total_bytes, bool ordered); + int64_t total_records, int64_t total_bytes, bool ordered, + std::string app_metadata); // ---------------------------------------------------------------------- // A pair of authentication handlers that check for a predefined password diff --git a/cpp/src/arrow/flight/types.cc b/cpp/src/arrow/flight/types.cc index a24a9ef8758ab..9da83fa8a11f2 100644 --- a/cpp/src/arrow/flight/types.cc +++ b/cpp/src/arrow/flight/types.cc @@ -260,13 +260,14 @@ arrow::Result FlightInfo::Make(const Schema& schema, const FlightDescriptor& descriptor, const std::vector& endpoints, int64_t total_records, int64_t total_bytes, - bool ordered) { + bool ordered, std::string app_metadata) { FlightInfo::Data data; data.descriptor = descriptor; data.endpoints = endpoints; data.total_records = total_records; data.total_bytes = total_bytes; data.ordered = ordered; + data.app_metadata = std::move(app_metadata); RETURN_NOT_OK(internal::SchemaToString(schema, &data.schema)); return FlightInfo(data); } @@ -328,6 +329,7 @@ std::string FlightInfo::ToString() const { ss << "] total_records=" << data_.total_records; ss << " total_bytes=" << data_.total_bytes; ss << " ordered=" << (data_.ordered ? "true" : "false"); + ss << " app_metadata='" << HexEncode(data_.app_metadata) << "'"; ss << '>'; return ss.str(); } @@ -338,7 +340,8 @@ bool FlightInfo::Equals(const FlightInfo& other) const { data_.endpoints == other.data_.endpoints && data_.total_records == other.data_.total_records && data_.total_bytes == other.data_.total_bytes && - data_.ordered == other.data_.ordered; + data_.ordered == other.data_.ordered && + data_.app_metadata == other.data_.app_metadata; } arrow::Result PollInfo::SerializeToString() const { @@ -535,6 +538,7 @@ std::string FlightEndpoint::ToString() const { } else { ss << "null"; } + ss << " app_metadata='" << HexEncode(app_metadata) << "'"; ss << ">"; return ss.str(); } @@ -554,6 +558,9 @@ bool FlightEndpoint::Equals(const FlightEndpoint& other) const { return false; } } + if (app_metadata != other.app_metadata) { + return false; + } return true; } diff --git a/cpp/src/arrow/flight/types.h b/cpp/src/arrow/flight/types.h index 70e7c90676b73..40a0787d14a7a 100644 --- a/cpp/src/arrow/flight/types.h +++ b/cpp/src/arrow/flight/types.h @@ -481,6 +481,9 @@ struct ARROW_FLIGHT_EXPORT FlightEndpoint { /// retrying DoGet requests. std::optional expiration_time; + /// Opaque Application-defined metadata + std::string app_metadata; + std::string ToString() const; bool Equals(const FlightEndpoint& other) const; @@ -583,6 +586,7 @@ class ARROW_FLIGHT_EXPORT FlightInfo { int64_t total_records = -1; int64_t total_bytes = -1; bool ordered = false; + std::string app_metadata; }; explicit FlightInfo(Data data) : data_(std::move(data)), reconstructed_schema_(false) {} @@ -592,7 +596,8 @@ class ARROW_FLIGHT_EXPORT FlightInfo { const FlightDescriptor& descriptor, const std::vector& endpoints, int64_t total_records, int64_t total_bytes, - bool ordered = false); + bool ordered = false, + std::string app_metadata = ""); /// \brief Deserialize the Arrow schema of the dataset. Populate any /// dictionary encoded fields into a DictionaryMemo for @@ -621,6 +626,9 @@ class ARROW_FLIGHT_EXPORT FlightInfo { /// Whether endpoints are in the same order as the data. bool ordered() const { return data_.ordered; } + /// Application-defined opaque metadata + const std::string& app_metadata() const { return data_.app_metadata; } + /// \brief Get the wire-format representation of this type. /// /// Useful when interoperating with non-Flight systems (e.g. REST diff --git a/dev/archery/archery/integration/runner.py b/dev/archery/archery/integration/runner.py index 67f0693df14aa..eb2e26951cd88 100644 --- a/dev/archery/archery/integration/runner.py +++ b/dev/archery/archery/integration/runner.py @@ -611,6 +611,11 @@ def run_all_tests(with_cpp=True, with_java=True, with_js=True, description="Ensure PollFlightInfo is supported.", skip_testers={"JS", "C#", "Rust"} ), + Scenario( + "app_metadata_flight_info_endpoint", + description="Ensure support FlightInfo and Endpoint app_metadata", + skip_testers={"JS", "C#", "Rust", "Java"} + ), Scenario( "flight_sql", description="Ensure Flight SQL protocol is working as expected.", diff --git a/format/Flight.proto b/format/Flight.proto index baf2733d81048..de3794f05ba83 100644 --- a/format/Flight.proto +++ b/format/Flight.proto @@ -366,6 +366,17 @@ message FlightInfo { * FlightEndpoints are in the same order as the data. */ bool ordered = 6; + + /* + * Application-defined metadata. + * + * There is no inherent or required relationship between this + * and the app_metadata fields in the FlightEndpoints or resulting + * FlightData messages. Since this metadata is application-defined, + * a given application could define there to be a relationship, + * but there is none required by the spec. + */ + bytes app_metadata = 7; } /* @@ -446,6 +457,17 @@ message FlightEndpoint { * application-defined whether DoGet requests may be retried. */ google.protobuf.Timestamp expiration_time = 3; + + /* + * Application-defined metadata. + * + * There is no inherent or required relationship between this + * and the app_metadata fields in the FlightInfo or resulting + * FlightData messages. Since this metadata is application-defined, + * a given application could define there to be a relationship, + * but there is none required by the spec. + */ + bytes app_metadata = 4; } /* diff --git a/go/arrow/array/numericbuilder.gen.go b/go/arrow/array/numericbuilder.gen.go index d3648f0b1fbc4..7f01180f55957 100644 --- a/go/arrow/array/numericbuilder.gen.go +++ b/go/arrow/array/numericbuilder.gen.go @@ -86,7 +86,7 @@ func (b *Int64Builder) AppendEmptyValue() { } func (b *Int64Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -564,7 +564,7 @@ func (b *Float64Builder) AppendEmptyValue() { } func (b *Float64Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -803,7 +803,7 @@ func (b *Int32Builder) AppendEmptyValue() { } func (b *Int32Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -1042,7 +1042,7 @@ func (b *Uint32Builder) AppendEmptyValue() { } func (b *Uint32Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -1281,7 +1281,7 @@ func (b *Float32Builder) AppendEmptyValue() { } func (b *Float32Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -1520,7 +1520,7 @@ func (b *Int16Builder) AppendEmptyValue() { } func (b *Int16Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -1759,7 +1759,7 @@ func (b *Uint16Builder) AppendEmptyValue() { } func (b *Uint16Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -1998,7 +1998,7 @@ func (b *Int8Builder) AppendEmptyValue() { } func (b *Int8Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -2237,7 +2237,7 @@ func (b *Uint8Builder) AppendEmptyValue() { } func (b *Uint8Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -2477,7 +2477,7 @@ func (b *Time32Builder) AppendEmptyValue() { } func (b *Time32Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -2717,7 +2717,7 @@ func (b *Time64Builder) AppendEmptyValue() { } func (b *Time64Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -2956,7 +2956,7 @@ func (b *Date32Builder) AppendEmptyValue() { } func (b *Date32Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -3195,7 +3195,7 @@ func (b *Date64Builder) AppendEmptyValue() { } func (b *Date64Builder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } @@ -3435,7 +3435,7 @@ func (b *DurationBuilder) AppendEmptyValue() { } func (b *DurationBuilder) AppendEmptyValues(n int) { - for i := 0; i < n; i ++ { + for i := 0; i < n; i++ { b.AppendEmptyValue() } } diff --git a/go/arrow/flight/gen.go b/go/arrow/flight/gen.go index cfdd0e036703a..29ae54b38f427 100644 --- a/go/arrow/flight/gen.go +++ b/go/arrow/flight/gen.go @@ -16,5 +16,5 @@ package flight -//go:generate protoc -I../../../format --go_out=./gen/flight --go-grpc_out=./gen/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=./gen/flight --go-grpc_out=./gen/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=./gen/flight --go-grpc_out=./gen/flight --go_opt=paths=source_relative --go-grpc_opt=paths=source_relative FlightSql.proto diff --git a/go/arrow/flight/gen/flight/Flight.pb.go b/go/arrow/flight/gen/flight/Flight.pb.go index 861b37d90d088..0438bca28be50 100644 --- a/go/arrow/flight/gen/flight/Flight.pb.go +++ b/go/arrow/flight/gen/flight/Flight.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.28.1 -// protoc v3.21.12 +// protoc v4.23.4 // source: Flight.proto package flight @@ -38,6 +38,7 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +// // The result of a cancel operation. // // This is used by CancelFlightInfoResult.status. @@ -102,17 +103,19 @@ func (CancelStatus) EnumDescriptor() ([]byte, []int) { return file_Flight_proto_rawDescGZIP(), []int{0} } +// // Describes what type of descriptor is defined. type FlightDescriptor_DescriptorType int32 const ( // Protobuf pattern, not used. FlightDescriptor_UNKNOWN FlightDescriptor_DescriptorType = 0 + // // A named path that identifies a dataset. A path is composed of a string // or list of strings describing a particular dataset. This is conceptually - // - // similar to a path inside a filesystem. + // similar to a path inside a filesystem. FlightDescriptor_PATH FlightDescriptor_DescriptorType = 1 + // // An opaque command to generate a dataset. FlightDescriptor_CMD FlightDescriptor_DescriptorType = 2 ) @@ -158,14 +161,17 @@ func (FlightDescriptor_DescriptorType) EnumDescriptor() ([]byte, []int) { return file_Flight_proto_rawDescGZIP(), []int{12, 0} } +// // The request that a client provides to a server on handshake. type HandshakeRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields + // // A defined protocol version ProtocolVersion uint64 `protobuf:"varint,1,opt,name=protocol_version,json=protocolVersion,proto3" json:"protocol_version,omitempty"` + // // Arbitrary auth/handshake info. Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` } @@ -221,8 +227,10 @@ type HandshakeResponse struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields + // // A defined protocol version ProtocolVersion uint64 `protobuf:"varint,1,opt,name=protocol_version,json=protocolVersion,proto3" json:"protocol_version,omitempty"` + // // Arbitrary auth/handshake info. Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` } @@ -273,6 +281,7 @@ func (x *HandshakeResponse) GetPayload() []byte { return nil } +// // A message for doing simple auth. type BasicAuth struct { state protoimpl.MessageState @@ -367,6 +376,7 @@ func (*Empty) Descriptor() ([]byte, []int) { return file_Flight_proto_rawDescGZIP(), []int{3} } +// // Describes an available action, including both the name used for execution // along with a short description of the purpose of the action. type ActionType struct { @@ -424,6 +434,7 @@ func (x *ActionType) GetDescription() string { return "" } +// // A service specific expression that can be used to return a limited set // of available Arrow Flight streams. type Criteria struct { @@ -473,6 +484,7 @@ func (x *Criteria) GetExpression() []byte { return nil } +// // An opaque action specific for the service. type Action struct { state protoimpl.MessageState @@ -529,6 +541,7 @@ func (x *Action) GetBody() []byte { return nil } +// // The request of the CancelFlightInfo action. // // The request should be stored in Action.body. @@ -579,6 +592,7 @@ func (x *CancelFlightInfoRequest) GetInfo() *FlightInfo { return nil } +// // The request of the RenewFlightEndpoint action. // // The request should be stored in Action.body. @@ -629,6 +643,7 @@ func (x *RenewFlightEndpointRequest) GetEndpoint() *FlightEndpoint { return nil } +// // An opaque result returned after executing an action. type Result struct { state protoimpl.MessageState @@ -677,6 +692,7 @@ func (x *Result) GetBody() []byte { return nil } +// // The result of the CancelFlightInfo action. // // The result should be stored in Result.body. @@ -727,6 +743,7 @@ func (x *CancelFlightInfoResult) GetStatus() CancelStatus { return CancelStatus_CANCEL_STATUS_UNSPECIFIED } +// // Wrap the result of a getSchema call type SchemaResult struct { state protoimpl.MessageState @@ -734,10 +751,9 @@ type SchemaResult struct { unknownFields protoimpl.UnknownFields // The schema of the dataset in its IPC form: - // - // 4 bytes - an optional IPC_CONTINUATION_TOKEN prefix - // 4 bytes - the byte length of the payload - // a flatbuffer Message whose header is the Schema + // 4 bytes - an optional IPC_CONTINUATION_TOKEN prefix + // 4 bytes - the byte length of the payload + // a flatbuffer Message whose header is the Schema Schema []byte `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"` } @@ -780,6 +796,7 @@ func (x *SchemaResult) GetSchema() []byte { return nil } +// // The name or tag for a Flight. May be used as a way to retrieve or generate // a flight or be used to expose a set of previously defined flights. type FlightDescriptor struct { @@ -788,9 +805,11 @@ type FlightDescriptor struct { unknownFields protoimpl.UnknownFields Type FlightDescriptor_DescriptorType `protobuf:"varint,1,opt,name=type,proto3,enum=arrow.flight.protocol.FlightDescriptor_DescriptorType" json:"type,omitempty"` + // // Opaque value used to express a command. Should only be defined when // type = CMD. Cmd []byte `protobuf:"bytes,2,opt,name=cmd,proto3" json:"cmd,omitempty"` + // // List of strings identifying a particular dataset. Should only be defined // when type = PATH. Path []string `protobuf:"bytes,3,rep,name=path,proto3" json:"path,omitempty"` @@ -849,6 +868,7 @@ func (x *FlightDescriptor) GetPath() []string { return nil } +// // The access coordinates for retrieval of a dataset. With a FlightInfo, a // consumer is able to determine how to retrieve a dataset. type FlightInfo struct { @@ -857,13 +877,14 @@ type FlightInfo struct { unknownFields protoimpl.UnknownFields // The schema of the dataset in its IPC form: - // - // 4 bytes - an optional IPC_CONTINUATION_TOKEN prefix - // 4 bytes - the byte length of the payload - // a flatbuffer Message whose header is the Schema + // 4 bytes - an optional IPC_CONTINUATION_TOKEN prefix + // 4 bytes - the byte length of the payload + // a flatbuffer Message whose header is the Schema Schema []byte `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"` + // // 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 (and hence all Tickets) must be // consumed. Endpoints can be consumed in any order. @@ -883,15 +904,19 @@ type FlightInfo struct { // ordering is important for an application, an application must // choose one of them: // - // - An application requires that all clients must read data in - // returned endpoints order. - // - An application must return the all data in a single endpoint. + // * An application requires that all clients must read data in + // returned endpoints order. + // * An application must return the all data 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"` TotalBytes int64 `protobuf:"varint,5,opt,name=total_bytes,json=totalBytes,proto3" json:"total_bytes,omitempty"` + // // FlightEndpoints are in the same order as the data. Ordered bool `protobuf:"varint,6,opt,name=ordered,proto3" json:"ordered,omitempty"` + // + // Application-defined metadata. + AppMetadata []byte `protobuf:"bytes,7,opt,name=app_metadata,json=appMetadata,proto3" json:"app_metadata,omitempty"` } func (x *FlightInfo) Reset() { @@ -968,12 +993,21 @@ func (x *FlightInfo) GetOrdered() bool { return false } +func (x *FlightInfo) GetAppMetadata() []byte { + if x != nil { + return x.AppMetadata + } + return nil +} + +// // The information to process a long-running query. type PollInfo struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields + // // The currently available results. // // If "flight_descriptor" is not specified, the query is complete @@ -991,12 +1025,15 @@ type PollInfo struct { // ticket in the info before the query is // completed. FlightInfo.ordered is also valid. Info *FlightInfo `protobuf:"bytes,1,opt,name=info,proto3" json:"info,omitempty"` + // // The descriptor the client should use on the next try. // If unset, the query is complete. FlightDescriptor *FlightDescriptor `protobuf:"bytes,2,opt,name=flight_descriptor,json=flightDescriptor,proto3" json:"flight_descriptor,omitempty"` + // // Query progress. If known, must be in [0.0, 1.0] but need not be // monotonic or nondecreasing. If unknown, do not set. Progress *float64 `protobuf:"fixed64,3,opt,name=progress,proto3,oneof" json:"progress,omitempty"` + // // Expiration time for this request. After this passes, the server // might not accept the retry descriptor anymore (and the query may // be cancelled). This may be updated on a call to PollFlightInfo. @@ -1063,14 +1100,17 @@ func (x *PollInfo) GetExpirationTime() *timestamppb.Timestamp { return nil } +// // A particular stream or split associated with a flight. type FlightEndpoint struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields + // // 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 via DoGet(). // // If the list is empty, the expectation is that the ticket can only @@ -1086,10 +1126,14 @@ type FlightEndpoint struct { // 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"` + // // Expiration time of this stream. If present, clients may assume // they can retry DoGet requests. Otherwise, it is // application-defined whether DoGet requests may be retried. ExpirationTime *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=expiration_time,json=expirationTime,proto3" json:"expiration_time,omitempty"` + // + // Application-defined metadata. + AppMetadata []byte `protobuf:"bytes,4,opt,name=app_metadata,json=appMetadata,proto3" json:"app_metadata,omitempty"` } func (x *FlightEndpoint) Reset() { @@ -1145,6 +1189,14 @@ func (x *FlightEndpoint) GetExpirationTime() *timestamppb.Timestamp { return nil } +func (x *FlightEndpoint) GetAppMetadata() []byte { + if x != nil { + return x.AppMetadata + } + return nil +} + +// // A location where a Flight service will accept retrieval of a particular // stream given a ticket. type Location struct { @@ -1194,6 +1246,7 @@ func (x *Location) GetUri() string { return "" } +// // An opaque identifier that the service can use to retrieve a particular // portion of a stream. // @@ -1246,19 +1299,24 @@ func (x *Ticket) GetTicket() []byte { return nil } +// // A batch of Arrow data as part of a stream of batches. type FlightData struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields + // // The descriptor of the data. This is only relevant when a client is // starting a new DoPut stream. FlightDescriptor *FlightDescriptor `protobuf:"bytes,1,opt,name=flight_descriptor,json=flightDescriptor,proto3" json:"flight_descriptor,omitempty"` + // // Header for message data as described in Message.fbs::Message. DataHeader []byte `protobuf:"bytes,2,opt,name=data_header,json=dataHeader,proto3" json:"data_header,omitempty"` + // // Application-defined metadata. AppMetadata []byte `protobuf:"bytes,3,opt,name=app_metadata,json=appMetadata,proto3" json:"app_metadata,omitempty"` + // // The actual batch of Arrow data. Preferably handled with minimal-copies // coming last in the definition to help with sidecar patterns (it is // expected that some implementations will fetch this field off the wire @@ -1326,7 +1384,7 @@ func (x *FlightData) GetDataBody() []byte { return nil } -// * +//* // The response message associated with the submission of a DoPut. type PutResult struct { state protoimpl.MessageState @@ -1441,7 +1499,7 @@ var file_Flight_proto_rawDesc = []byte{ 0x22, 0x30, 0x0a, 0x0e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x50, 0x41, 0x54, 0x48, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x43, 0x4d, 0x44, - 0x10, 0x02, 0x22, 0x9d, 0x02, 0x0a, 0x0a, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x49, 0x6e, 0x66, + 0x10, 0x02, 0x22, 0xc0, 0x02, 0x0a, 0x0a, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x54, 0x0a, 0x11, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x18, 0x02, @@ -1459,129 +1517,133 @@ var file_Flight_proto_rawDesc = []byte{ 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x6f, 0x72, 0x64, 0x65, 0x72, - 0x65, 0x64, 0x22, 0x8a, 0x02, 0x0a, 0x08, 0x50, 0x6f, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, - 0x35, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, - 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x49, 0x6e, 0x66, 0x6f, - 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x12, 0x54, 0x0a, 0x11, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, - 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x27, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, - 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x10, 0x66, 0x6c, 0x69, 0x67, - 0x68, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, 0x1f, 0x0a, 0x08, - 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, - 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x88, 0x01, 0x01, 0x12, 0x43, 0x0a, - 0x0f, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, - 0x6d, 0x70, 0x52, 0x0e, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, - 0x6d, 0x65, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x22, - 0xc9, 0x01, 0x0a, 0x0e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, - 0x6e, 0x74, 0x12, 0x35, 0x0a, 0x06, 0x74, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, - 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, - 0x74, 0x52, 0x06, 0x74, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x12, 0x3b, 0x0a, 0x08, 0x6c, 0x6f, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x61, 0x72, + 0x65, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x70, 0x70, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x61, 0x70, 0x70, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x22, 0x8a, 0x02, 0x0a, 0x08, 0x50, 0x6f, 0x6c, 0x6c, 0x49, 0x6e, + 0x66, 0x6f, 0x12, 0x35, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x21, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x12, 0x54, 0x0a, 0x11, 0x66, 0x6c, 0x69, + 0x67, 0x68, 0x74, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, + 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x46, 0x6c, 0x69, + 0x67, 0x68, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x10, 0x66, + 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, + 0x1f, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x01, 0x48, 0x00, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x88, 0x01, 0x01, + 0x12, 0x43, 0x0a, 0x0f, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, + 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0e, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, + 0x73, 0x73, 0x22, 0xec, 0x01, 0x0a, 0x0e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x45, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x35, 0x0a, 0x06, 0x74, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, + 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x54, 0x69, + 0x63, 0x6b, 0x65, 0x74, 0x52, 0x06, 0x74, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x12, 0x3b, 0x0a, 0x08, + 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, + 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x43, 0x0a, 0x0f, 0x65, 0x78, 0x70, + 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0e, + 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x21, + 0x0a, 0x0c, 0x61, 0x70, 0x70, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x61, 0x70, 0x70, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x22, 0x1c, 0x0a, 0x08, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, + 0x03, 0x75, 0x72, 0x69, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x69, 0x22, + 0x20, 0x0a, 0x06, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x69, 0x63, + 0x6b, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x74, 0x69, 0x63, 0x6b, 0x65, + 0x74, 0x22, 0xc4, 0x01, 0x0a, 0x0a, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x61, 0x74, 0x61, + 0x12, 0x54, 0x0a, 0x11, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x63, 0x6f, 0x6c, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x6c, 0x6f, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x43, 0x0a, 0x0f, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0e, 0x65, 0x78, 0x70, - 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x1c, 0x0a, 0x08, 0x4c, - 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x69, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x69, 0x22, 0x20, 0x0a, 0x06, 0x54, 0x69, 0x63, - 0x6b, 0x65, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0c, 0x52, 0x06, 0x74, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x22, 0xc4, 0x01, 0x0a, 0x0a, - 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x61, 0x74, 0x61, 0x12, 0x54, 0x0a, 0x11, 0x66, 0x6c, - 0x69, 0x67, 0x68, 0x74, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, - 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x46, 0x6c, - 0x69, 0x67, 0x68, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x10, - 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, - 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x48, 0x65, 0x61, 0x64, 0x65, - 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x70, 0x70, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, - 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x61, 0x70, 0x70, 0x4d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0x12, 0x1c, 0x0a, 0x09, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x62, 0x6f, 0x64, - 0x79, 0x18, 0xe8, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x42, 0x6f, - 0x64, 0x79, 0x22, 0x2e, 0x0a, 0x09, 0x50, 0x75, 0x74, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, - 0x21, 0x0a, 0x0c, 0x61, 0x70, 0x70, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x61, 0x70, 0x70, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x2a, 0x8b, 0x01, 0x0a, 0x0c, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x12, 0x1d, 0x0a, 0x19, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x53, 0x54, - 0x41, 0x54, 0x55, 0x53, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, - 0x10, 0x00, 0x12, 0x1b, 0x0a, 0x17, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x53, 0x54, 0x41, - 0x54, 0x55, 0x53, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x01, 0x12, - 0x1c, 0x0a, 0x18, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, - 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x21, 0x0a, - 0x1d, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x4e, - 0x4f, 0x54, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x03, - 0x32, 0x85, 0x07, 0x0a, 0x0d, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x12, 0x64, 0x0a, 0x09, 0x48, 0x61, 0x6e, 0x64, 0x73, 0x68, 0x61, 0x6b, 0x65, 0x12, + 0x63, 0x6f, 0x6c, 0x2e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x6f, 0x72, 0x52, 0x10, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x68, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x64, 0x61, 0x74, + 0x61, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x70, 0x70, 0x5f, 0x6d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x61, + 0x70, 0x70, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1c, 0x0a, 0x09, 0x64, 0x61, + 0x74, 0x61, 0x5f, 0x62, 0x6f, 0x64, 0x79, 0x18, 0xe8, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, + 0x64, 0x61, 0x74, 0x61, 0x42, 0x6f, 0x64, 0x79, 0x22, 0x2e, 0x0a, 0x09, 0x50, 0x75, 0x74, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x70, 0x70, 0x5f, 0x6d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x61, 0x70, 0x70, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x2a, 0x8b, 0x01, 0x0a, 0x0c, 0x43, 0x61, 0x6e, + 0x63, 0x65, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1d, 0x0a, 0x19, 0x43, 0x41, 0x4e, + 0x43, 0x45, 0x4c, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, + 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1b, 0x0a, 0x17, 0x43, 0x41, 0x4e, 0x43, + 0x45, 0x4c, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, + 0x4c, 0x45, 0x44, 0x10, 0x01, 0x12, 0x1c, 0x0a, 0x18, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, + 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x49, 0x4e, + 0x47, 0x10, 0x02, 0x12, 0x21, 0x0a, 0x1d, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x53, 0x54, + 0x41, 0x54, 0x55, 0x53, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, + 0x41, 0x42, 0x4c, 0x45, 0x10, 0x03, 0x32, 0x85, 0x07, 0x0a, 0x0d, 0x46, 0x6c, 0x69, 0x67, 0x68, + 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x64, 0x0a, 0x09, 0x48, 0x61, 0x6e, 0x64, + 0x73, 0x68, 0x61, 0x6b, 0x65, 0x12, 0x27, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, + 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x48, 0x61, + 0x6e, 0x64, 0x73, 0x68, 0x61, 0x6b, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, + 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x48, 0x61, 0x6e, 0x64, 0x73, 0x68, 0x61, 0x6b, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x55, + 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x73, 0x12, 0x1f, 0x2e, + 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x43, 0x72, 0x69, 0x74, 0x65, 0x72, 0x69, 0x61, 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, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x49, 0x6e, 0x66, + 0x6f, 0x22, 0x00, 0x30, 0x01, 0x12, 0x5d, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x69, 0x67, + 0x68, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x27, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, + 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x46, + 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 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, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x49, 0x6e, + 0x66, 0x6f, 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x0e, 0x50, 0x6f, 0x6c, 0x6c, 0x46, 0x6c, 0x69, 0x67, + 0x68, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x27, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, + 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x46, + 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x1a, + 0x1f, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x50, 0x6f, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, + 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x27, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x48, 0x61, 0x6e, 0x64, 0x73, 0x68, 0x61, 0x6b, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, + 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x1a, 0x23, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, - 0x2e, 0x48, 0x61, 0x6e, 0x64, 0x73, 0x68, 0x61, 0x6b, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x55, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, - 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x73, 0x12, 0x1f, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, - 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, - 0x43, 0x72, 0x69, 0x74, 0x65, 0x72, 0x69, 0x61, 0x1a, 0x21, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, + 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x00, 0x12, + 0x4d, 0x0a, 0x05, 0x44, 0x6f, 0x47, 0x65, 0x74, 0x12, 0x1d, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, - 0x2e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x00, 0x30, 0x01, 0x12, - 0x5d, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x49, 0x6e, 0x66, 0x6f, - 0x12, 0x27, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, - 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 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, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x00, 0x12, 0x5c, - 0x0a, 0x0e, 0x50, 0x6f, 0x6c, 0x6c, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x49, 0x6e, 0x66, 0x6f, - 0x12, 0x27, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, + 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 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, + 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x61, 0x74, 0x61, 0x22, 0x00, 0x30, 0x01, 0x12, 0x52, + 0x0a, 0x05, 0x44, 0x6f, 0x50, 0x75, 0x74, 0x12, 0x21, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, + 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, + 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x61, 0x74, 0x61, 0x1a, 0x20, 0x2e, 0x61, 0x72, 0x72, + 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, + 0x6f, 0x6c, 0x2e, 0x50, 0x75, 0x74, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x00, 0x28, 0x01, + 0x30, 0x01, 0x12, 0x58, 0x0a, 0x0a, 0x44, 0x6f, 0x45, 0x78, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, + 0x12, 0x21, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, - 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x1a, 0x1f, 0x2e, 0x61, 0x72, 0x72, 0x6f, - 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, - 0x6c, 0x2e, 0x50, 0x6f, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x09, - 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x27, 0x2e, 0x61, 0x72, 0x72, 0x6f, + 0x61, 0x74, 0x61, 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, 0x46, 0x6c, 0x69, 0x67, + 0x68, 0x74, 0x44, 0x61, 0x74, 0x61, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x4c, 0x0a, 0x08, + 0x44, 0x6f, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 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, 0x1a, 0x1d, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, + 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, + 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x00, 0x30, 0x01, 0x12, 0x52, 0x0a, 0x0b, 0x4c, 0x69, + 0x73, 0x74, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1c, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, - 0x6c, 0x2e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, - 0x6f, 0x72, 0x1a, 0x23, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, - 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x00, 0x12, 0x4d, 0x0a, 0x05, 0x44, 0x6f, 0x47, - 0x65, 0x74, 0x12, 0x1d, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, - 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, - 0x74, 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, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, - 0x44, 0x61, 0x74, 0x61, 0x22, 0x00, 0x30, 0x01, 0x12, 0x52, 0x0a, 0x05, 0x44, 0x6f, 0x50, 0x75, - 0x74, 0x12, 0x21, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, - 0x44, 0x61, 0x74, 0x61, 0x1a, 0x20, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, - 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x50, 0x75, 0x74, - 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x58, 0x0a, 0x0a, - 0x44, 0x6f, 0x45, 0x78, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x21, 0x2e, 0x61, 0x72, 0x72, - 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, - 0x6f, 0x6c, 0x2e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x61, 0x74, 0x61, 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, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x44, 0x61, 0x74, 0x61, - 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x4c, 0x0a, 0x08, 0x44, 0x6f, 0x41, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x1d, 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, 0x1a, 0x1d, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, - 0x22, 0x00, 0x30, 0x01, 0x12, 0x52, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x12, 0x1c, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, - 0x68, 0x74, 0x2e, 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, 0x71, 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, 0x32, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, - 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x61, 0x72, 0x72, 0x6f, - 0x77, 0x2f, 0x67, 0x6f, 0x2f, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2f, 0x66, 0x6c, 0x69, 0x67, 0x68, - 0x74, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 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, + 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, 0x71, + 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, 0x32, + 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2f, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2f, 0x67, 0x6f, 0x2f, 0x61, 0x72, 0x72, 0x6f, 0x77, + 0x2f, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x66, 0x6c, 0x69, 0x67, + 0x68, 0x74, 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/gen/flight/FlightSql.pb.go b/go/arrow/flight/gen/flight/FlightSql.pb.go index 95faa719a9488..494bf8bcca115 100644 --- a/go/arrow/flight/gen/flight/FlightSql.pb.go +++ b/go/arrow/flight/gen/flight/FlightSql.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.28.1 -// protoc v3.21.12 +// protoc v4.23.4 // source: FlightSql.proto package flight @@ -48,27 +48,33 @@ const ( 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 executing // SQL queries. // // Note that the absence of this info (as opposed to a false value) does not necessarily // mean that SQL is not supported, as this property was not originally defined. SqlInfo_FLIGHT_SQL_SERVER_SQL SqlInfo = 4 + // // Retrieves a boolean value indicating whether the Flight SQL Server supports executing // Substrait plans. SqlInfo_FLIGHT_SQL_SERVER_SUBSTRAIT SqlInfo = 5 + // // Retrieves a string value indicating the minimum supported Substrait version, or null // if Substrait is not supported. SqlInfo_FLIGHT_SQL_SERVER_SUBSTRAIT_MIN_VERSION SqlInfo = 6 + // // Retrieves a string value indicating the maximum supported Substrait version, or null // if Substrait is not supported. SqlInfo_FLIGHT_SQL_SERVER_SUBSTRAIT_MAX_VERSION SqlInfo = 7 + // // Retrieves an int32 indicating whether the Flight SQL Server supports the // BeginTransaction/EndTransaction/BeginSavepoint/EndSavepoint actions. // @@ -78,51 +84,61 @@ const ( // // The possible values are listed in `SqlSupportedTransaction`. SqlInfo_FLIGHT_SQL_SERVER_TRANSACTION SqlInfo = 8 + // // Retrieves a boolean value indicating whether the Flight SQL Server supports explicit // query cancellation (the CancelQuery action). SqlInfo_FLIGHT_SQL_SERVER_CANCEL SqlInfo = 9 + // // Retrieves an int32 indicating the timeout (in milliseconds) for prepared statement handles. // // If 0, there is no timeout. Servers should reset the timeout when the handle is used in a command. SqlInfo_FLIGHT_SQL_SERVER_STATEMENT_TIMEOUT SqlInfo = 100 + // // Retrieves an int32 indicating the timeout (in milliseconds) for transactions, since transactions are not tied to a connection. // // If 0, there is no timeout. Servers should reset the timeout when the handle is used in a command. SqlInfo_FLIGHT_SQL_SERVER_TRANSACTION_TIMEOUT SqlInfo = 101 + // // 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 @@ -138,15 +154,18 @@ const ( 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. @@ -155,6 +174,7 @@ const ( // - 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. // @@ -162,11 +182,13 @@ const ( // - 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. // @@ -174,6 +196,7 @@ const ( // - 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. // @@ -181,12 +204,14 @@ const ( // - 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. // @@ -194,6 +219,7 @@ const ( // - 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. @@ -206,18 +232,21 @@ const ( // - 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. @@ -234,6 +263,7 @@ const ( // - 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. @@ -250,12 +280,14 @@ const ( // - 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 @@ -265,14 +297,17 @@ const ( 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. @@ -289,6 +324,7 @@ const ( // - 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. @@ -305,6 +341,7 @@ const ( // - 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. @@ -317,12 +354,14 @@ const ( // - 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. // @@ -330,6 +369,7 @@ const ( // - 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. @@ -355,12 +395,14 @@ const ( // - ... // 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. @@ -393,6 +435,7 @@ const ( 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 @@ -404,15 +447,17 @@ const ( 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. + // - 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 @@ -424,11 +469,13 @@ const ( 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`. // @@ -436,6 +483,7 @@ const ( // - 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. @@ -462,6 +510,7 @@ const ( // - ... // 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. // @@ -469,12 +518,14 @@ const ( // - 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. // @@ -491,6 +542,7 @@ const ( // - ... // 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`. // @@ -505,6 +557,7 @@ const ( // - 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`. // @@ -519,6 +572,7 @@ const ( // - 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`. // @@ -533,6 +587,7 @@ const ( // - 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`. // @@ -547,29 +602,34 @@ const ( // - 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. // @@ -1642,7 +1702,7 @@ func (SqlSupportsConvert) EnumDescriptor() ([]byte, []int) { return file_FlightSql_proto_rawDescGZIP(), []int{16} } -// * +//* // 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 @@ -1757,7 +1817,7 @@ func (XdbcDataType) EnumDescriptor() ([]byte, []int) { return file_FlightSql_proto_rawDescGZIP(), []int{17} } -// * +//* // Detailed subtype information for XDBC_TYPE_DATETIME and XDBC_TYPE_INTERVAL. type XdbcDatetimeSubcode int32 @@ -1898,13 +1958,13 @@ func (XdbcDatetimeSubcode) EnumDescriptor() ([]byte, []int) { 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 ) @@ -1953,21 +2013,21 @@ func (Nullable) EnumDescriptor() ([]byte, []int) { 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. + // - 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 ) @@ -2233,23 +2293,22 @@ func (ActionCancelQueryResult_CancelResult) EnumDescriptor() ([]byte, []int) { return file_FlightSql_proto_rawDescGZIP(), []int{29, 0} } +// // 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. +// - 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> -// +// 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 { @@ -2257,6 +2316,7 @@ type CommandGetSqlInfo struct { 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. @@ -2316,62 +2376,61 @@ func (x *CommandGetSqlInfo) GetInfo() []uint32 { 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. +// - 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: int32 not null (The SQL data type), -// column_size: int32 (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: int32 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: int32 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: int32 (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: int32 (The maximum scale of the data type on the data source. -// NULL is returned if scale is not applicable.), -// sql_data_type: int32 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: int32 (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: int32 (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: int32 (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). -// +// type_name: utf8 not null (The name of the data type, for example: VARCHAR, INTEGER, etc), +// data_type: int32 not null (The SQL data type), +// column_size: int32 (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: int32 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: int32 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: int32 (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: int32 (The maximum scale of the data type on the data source. +// NULL is returned if scale is not applicable.), +// sql_data_type: int32 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: int32 (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: int32 (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: int32 (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 { @@ -2379,6 +2438,7 @@ type CommandGetXdbcTypeInfo struct { 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"` } @@ -2422,17 +2482,16 @@ func (x *CommandGetXdbcTypeInfo) GetDataType() int32 { 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. +// - 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 -// +// catalog_name: utf8 not null // > // The returned data should be ordered by catalog_name. type CommandGetCatalogs struct { @@ -2473,18 +2532,17 @@ 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. +// - 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 -// +// 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 { @@ -2492,15 +2550,17 @@ type CommandGetDbSchemas struct { 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. + // - "%" 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"` } @@ -2550,56 +2610,58 @@ func (x *CommandGetDbSchemas) GetDbSchemaFilterPattern() string { 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. +// - 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.) -// +// 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. -// +// - 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. + // - "%" 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. + // - "%" 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. @@ -2675,18 +2737,17 @@ func (x *CommandGetTables) GetIncludeSchema() bool { 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. +// - 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 -// +// table_type: utf8 not null // > // The returned data should be ordered by table_type. type CommandGetTableTypes struct { @@ -2727,21 +2788,20 @@ 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. +// - 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: int32 not null -// +// catalog_name: utf8, +// db_schema_name: utf8, +// table_name: utf8 not null, +// column_name: utf8 not null, +// key_name: utf8, +// key_sequence: int32 not null // > // The returned data should be ordered by catalog_name, db_schema_name, table_name, key_name, then key_sequence. type CommandGetPrimaryKeys struct { @@ -2749,10 +2809,12 @@ type CommandGetPrimaryKeys struct { 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. @@ -2814,29 +2876,28 @@ func (x *CommandGetPrimaryKeys) GetTable() string { 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. +// - 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: int32 not null, -// fk_key_name: utf8, -// pk_key_name: utf8, -// update_rule: uint8 not null, -// delete_rule: uint8 not null -// +// 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: int32 not null, +// fk_key_name: utf8, +// pk_key_name: utf8, +// update_rule: uint8 not null, +// delete_rule: uint8 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. @@ -2845,10 +2906,12 @@ type CommandGetExportedKeys struct { 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. @@ -2910,45 +2973,46 @@ func (x *CommandGetExportedKeys) GetTable() string { 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. +// - 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: int32 not null, -// fk_key_name: utf8, -// pk_key_name: utf8, -// update_rule: uint8 not null, -// delete_rule: uint8 not null -// +// 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: int32 not null, +// fk_key_name: utf8, +// pk_key_name: utf8, +// update_rule: uint8 not null, +// delete_rule: uint8 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 +// - 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. @@ -3010,67 +3074,66 @@ func (x *CommandGetImportedKeys) GetTable() string { 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. +// - 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: int32 not null, -// fk_key_name: utf8, -// pk_key_name: utf8, -// update_rule: uint8 not null, -// delete_rule: uint8 not null -// +// 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: int32 not null, +// fk_key_name: utf8, +// pk_key_name: utf8, +// update_rule: uint8 not null, +// delete_rule: uint8 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 +// - 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"` } @@ -3149,6 +3212,7 @@ func (x *CommandGetCrossReference) GetFkTable() string { return "" } +// // Request message for the "CreatePreparedStatement" action on a Flight SQL enabled backend. type ActionCreatePreparedStatementRequest struct { state protoimpl.MessageState @@ -3208,6 +3272,7 @@ func (x *ActionCreatePreparedStatementRequest) GetTransactionId() []byte { return nil } +// // An embedded message describing a Substrait plan to execute. type SubstraitPlan struct { state protoimpl.MessageState @@ -3271,6 +3336,7 @@ func (x *SubstraitPlan) GetVersion() string { return "" } +// // Request message for the "CreatePreparedSubstraitPlan" action on a Flight SQL enabled backend. type ActionCreatePreparedSubstraitPlanRequest struct { state protoimpl.MessageState @@ -3330,6 +3396,7 @@ func (x *ActionCreatePreparedSubstraitPlanRequest) GetTransactionId() []byte { return nil } +// // Wrap the result of a "CreatePreparedStatement" or "CreatePreparedSubstraitPlan" action. // // The resultant PreparedStatement can be closed either: @@ -3405,6 +3472,7 @@ func (x *ActionCreatePreparedStatementResult) GetParameterSchema() []byte { 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 { @@ -3455,6 +3523,7 @@ func (x *ActionClosePreparedStatementRequest) GetPreparedStatementHandle() []byt return nil } +// // Request message for the "BeginTransaction" action. // Begins a transaction. type ActionBeginTransactionRequest struct { @@ -3495,6 +3564,7 @@ func (*ActionBeginTransactionRequest) Descriptor() ([]byte, []int) { return file_FlightSql_proto_rawDescGZIP(), []int{15} } +// // Request message for the "BeginSavepoint" action. // Creates a savepoint within a transaction. // @@ -3557,6 +3627,7 @@ func (x *ActionBeginSavepointRequest) GetName() string { return "" } +// // The result of a "BeginTransaction" action. // // The transaction can be manipulated with the "EndTransaction" action, or @@ -3612,6 +3683,7 @@ func (x *ActionBeginTransactionResult) GetTransactionId() []byte { return nil } +// // The result of a "BeginSavepoint" action. // // The transaction can be manipulated with the "EndSavepoint" action. @@ -3667,6 +3739,7 @@ func (x *ActionBeginSavepointResult) GetSavepointId() []byte { return nil } +// // Request message for the "EndTransaction" action. // // Commit (COMMIT) or rollback (ROLLBACK) the transaction. @@ -3730,6 +3803,7 @@ func (x *ActionEndTransactionRequest) GetAction() ActionEndTransactionRequest_En return ActionEndTransactionRequest_END_TRANSACTION_UNSPECIFIED } +// // Request message for the "EndSavepoint" action. // // Release (RELEASE) the savepoint or rollback (ROLLBACK) to the @@ -3795,21 +3869,22 @@ func (x *ActionEndSavepointRequest) GetAction() ActionEndSavepointRequest_EndSav return ActionEndSavepointRequest_END_SAVEPOINT_UNSPECIFIED } +// // 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. +// - 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 @@ -3867,22 +3942,23 @@ func (x *CommandStatementQuery) GetTransactionId() []byte { return nil } +// // Represents a Substrait plan. 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. -// - DoPut: execute the query. +// - 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. +// - DoPut: execute the query. type CommandStatementSubstraitPlan struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3940,7 +4016,7 @@ func (x *CommandStatementSubstraitPlan) GetTransactionId() []byte { return nil } -// * +//* // 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 { @@ -3991,22 +4067,23 @@ func (x *TicketStatementQuery) GetStatementHandle() []byte { 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. +// - 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 @@ -4055,6 +4132,7 @@ func (x *CommandPreparedStatementQuery) GetPreparedStatementHandle() []byte { 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 { @@ -4114,6 +4192,7 @@ func (x *CommandStatementUpdate) GetTransactionId() []byte { 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 // prepared statement handle as an update. @@ -4165,6 +4244,7 @@ func (x *CommandPreparedStatementUpdate) GetPreparedStatementHandle() []byte { return nil } +// // Returned from the RPC call DoPut when a CommandStatementUpdate // CommandPreparedStatementUpdate was in the request, containing // results from the update. @@ -4217,6 +4297,7 @@ func (x *DoPutUpdateResult) GetRecordCount() int64 { return 0 } +// // Request message for the "CancelQuery" action. // // Explicitly cancel a running query. @@ -4285,6 +4366,7 @@ func (x *ActionCancelQueryRequest) GetInfo() []byte { return nil } +// // The result of cancelling a query. // // The result should be wrapped in a google.protobuf.Any message. diff --git a/go/arrow/flight/gen/flight/Flight_grpc.pb.go b/go/arrow/flight/gen/flight/Flight_grpc.pb.go index 150de71a1113b..87d9abc5926eb 100644 --- a/go/arrow/flight/gen/flight/Flight_grpc.pb.go +++ b/go/arrow/flight/gen/flight/Flight_grpc.pb.go @@ -1,4 +1,8 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.2.0 +// - protoc v4.23.4 +// source: Flight.proto package flight @@ -11,17 +15,20 @@ import ( // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. const _ = grpc.SupportPackageIsVersion7 // FlightServiceClient is the client API for FlightService service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. type FlightServiceClient interface { + // // Handshake between client and server. Depending on the server, the // handshake may be required to determine the token that should be used for // future operations. Both request and response are streams to allow multiple // round-trips depending on auth mechanism. Handshake(ctx context.Context, opts ...grpc.CallOption) (FlightService_HandshakeClient, error) + // // Get a list of available streams given a particular criteria. Most flight // services will expose one or more streams that are readily available for // retrieval. This api allows listing the streams available for @@ -29,6 +36,7 @@ type FlightServiceClient interface { // the subset of streams that can be listed via this interface. Each flight // service allows its own definition of how to consume criteria. ListFlights(ctx context.Context, in *Criteria, opts ...grpc.CallOption) (FlightService_ListFlightsClient, error) + // // For a given FlightDescriptor, get information about how the flight can be // consumed. This is a useful interface if the consumer of the interface // already can identify the specific flight to consume. This interface can @@ -40,6 +48,7 @@ type FlightServiceClient interface { // available for consumption for the duration defined by the specific flight // service. GetFlightInfo(ctx context.Context, in *FlightDescriptor, opts ...grpc.CallOption) (*FlightInfo, error) + // // For a given FlightDescriptor, start a query and get information // to poll its execution status. This is a useful interface if the // query may be a long-running query. The first PollFlightInfo call @@ -63,16 +72,19 @@ type FlightServiceClient interface { // A client may use the CancelFlightInfo action with // PollInfo.info to cancel the running query. PollFlightInfo(ctx context.Context, in *FlightDescriptor, opts ...grpc.CallOption) (*PollInfo, error) + // // For a given FlightDescriptor, get the Schema as described in Schema.fbs::Schema // This is used when a consumer needs the Schema of flight stream. Similar to // GetFlightInfo this interface may generate a new flight that was not previously // available in ListFlights. GetSchema(ctx context.Context, in *FlightDescriptor, opts ...grpc.CallOption) (*SchemaResult, error) + // // Retrieve a single stream associated with a particular descriptor // associated with the referenced ticket. A Flight can be composed of one or // more streams where each stream can be retrieved using a separate opaque // ticket that the flight service uses for managing a collection of streams. DoGet(ctx context.Context, in *Ticket, opts ...grpc.CallOption) (FlightService_DoGetClient, error) + // // Push a stream to the flight service associated with a particular // flight stream. This allows a client of a flight service to upload a stream // of data. Depending on the particular flight service, a client consumer @@ -80,12 +92,14 @@ type FlightServiceClient interface { // number. In the latter, the service might implement a 'seal' action that // can be applied to a descriptor once all streams are uploaded. DoPut(ctx context.Context, opts ...grpc.CallOption) (FlightService_DoPutClient, error) + // // Open a bidirectional data channel for a given descriptor. This // allows clients to send and receive arbitrary Arrow data and // application-specific metadata in a single logical stream. In // contrast to DoGet/DoPut, this is more suited for clients // offloading computation (rather than storage) to a Flight service. DoExchange(ctx context.Context, opts ...grpc.CallOption) (FlightService_DoExchangeClient, error) + // // Flight services can support an arbitrary number of simple actions in // addition to the possible ListFlights, GetFlightInfo, DoGet, DoPut // operations that are potentially available. DoAction allows a flight client @@ -93,6 +107,7 @@ type FlightServiceClient interface { // opaque request and response objects that are specific to the type action // being undertaken. DoAction(ctx context.Context, in *Action, opts ...grpc.CallOption) (FlightService_DoActionClient, error) + // // A flight service exposes all of the available action types that it has // along with descriptions. This allows different flight consumers to // understand the capabilities of the flight service. @@ -108,7 +123,7 @@ func NewFlightServiceClient(cc grpc.ClientConnInterface) FlightServiceClient { } func (c *flightServiceClient) Handshake(ctx context.Context, opts ...grpc.CallOption) (FlightService_HandshakeClient, error) { - stream, err := c.cc.NewStream(ctx, &_FlightService_serviceDesc.Streams[0], "/arrow.flight.protocol.FlightService/Handshake", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[0], "/arrow.flight.protocol.FlightService/Handshake", opts...) if err != nil { return nil, err } @@ -139,7 +154,7 @@ func (x *flightServiceHandshakeClient) Recv() (*HandshakeResponse, error) { } func (c *flightServiceClient) ListFlights(ctx context.Context, in *Criteria, opts ...grpc.CallOption) (FlightService_ListFlightsClient, error) { - stream, err := c.cc.NewStream(ctx, &_FlightService_serviceDesc.Streams[1], "/arrow.flight.protocol.FlightService/ListFlights", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[1], "/arrow.flight.protocol.FlightService/ListFlights", opts...) if err != nil { return nil, err } @@ -198,7 +213,7 @@ func (c *flightServiceClient) GetSchema(ctx context.Context, in *FlightDescripto } func (c *flightServiceClient) DoGet(ctx context.Context, in *Ticket, opts ...grpc.CallOption) (FlightService_DoGetClient, error) { - stream, err := c.cc.NewStream(ctx, &_FlightService_serviceDesc.Streams[2], "/arrow.flight.protocol.FlightService/DoGet", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[2], "/arrow.flight.protocol.FlightService/DoGet", opts...) if err != nil { return nil, err } @@ -230,7 +245,7 @@ func (x *flightServiceDoGetClient) Recv() (*FlightData, error) { } func (c *flightServiceClient) DoPut(ctx context.Context, opts ...grpc.CallOption) (FlightService_DoPutClient, error) { - stream, err := c.cc.NewStream(ctx, &_FlightService_serviceDesc.Streams[3], "/arrow.flight.protocol.FlightService/DoPut", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[3], "/arrow.flight.protocol.FlightService/DoPut", opts...) if err != nil { return nil, err } @@ -261,7 +276,7 @@ func (x *flightServiceDoPutClient) Recv() (*PutResult, error) { } func (c *flightServiceClient) DoExchange(ctx context.Context, opts ...grpc.CallOption) (FlightService_DoExchangeClient, error) { - stream, err := c.cc.NewStream(ctx, &_FlightService_serviceDesc.Streams[4], "/arrow.flight.protocol.FlightService/DoExchange", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[4], "/arrow.flight.protocol.FlightService/DoExchange", opts...) if err != nil { return nil, err } @@ -292,7 +307,7 @@ func (x *flightServiceDoExchangeClient) Recv() (*FlightData, error) { } func (c *flightServiceClient) DoAction(ctx context.Context, in *Action, opts ...grpc.CallOption) (FlightService_DoActionClient, error) { - stream, err := c.cc.NewStream(ctx, &_FlightService_serviceDesc.Streams[5], "/arrow.flight.protocol.FlightService/DoAction", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[5], "/arrow.flight.protocol.FlightService/DoAction", opts...) if err != nil { return nil, err } @@ -324,7 +339,7 @@ func (x *flightServiceDoActionClient) Recv() (*Result, error) { } func (c *flightServiceClient) ListActions(ctx context.Context, in *Empty, opts ...grpc.CallOption) (FlightService_ListActionsClient, error) { - stream, err := c.cc.NewStream(ctx, &_FlightService_serviceDesc.Streams[6], "/arrow.flight.protocol.FlightService/ListActions", opts...) + stream, err := c.cc.NewStream(ctx, &FlightService_ServiceDesc.Streams[6], "/arrow.flight.protocol.FlightService/ListActions", opts...) if err != nil { return nil, err } @@ -359,11 +374,13 @@ func (x *flightServiceListActionsClient) Recv() (*ActionType, error) { // All implementations must embed UnimplementedFlightServiceServer // for forward compatibility type FlightServiceServer interface { + // // Handshake between client and server. Depending on the server, the // handshake may be required to determine the token that should be used for // future operations. Both request and response are streams to allow multiple // round-trips depending on auth mechanism. Handshake(FlightService_HandshakeServer) error + // // Get a list of available streams given a particular criteria. Most flight // services will expose one or more streams that are readily available for // retrieval. This api allows listing the streams available for @@ -371,6 +388,7 @@ type FlightServiceServer interface { // the subset of streams that can be listed via this interface. Each flight // service allows its own definition of how to consume criteria. ListFlights(*Criteria, FlightService_ListFlightsServer) error + // // For a given FlightDescriptor, get information about how the flight can be // consumed. This is a useful interface if the consumer of the interface // already can identify the specific flight to consume. This interface can @@ -382,6 +400,7 @@ type FlightServiceServer interface { // available for consumption for the duration defined by the specific flight // service. GetFlightInfo(context.Context, *FlightDescriptor) (*FlightInfo, error) + // // For a given FlightDescriptor, start a query and get information // to poll its execution status. This is a useful interface if the // query may be a long-running query. The first PollFlightInfo call @@ -405,16 +424,19 @@ type FlightServiceServer interface { // A client may use the CancelFlightInfo action with // PollInfo.info to cancel the running query. PollFlightInfo(context.Context, *FlightDescriptor) (*PollInfo, error) + // // For a given FlightDescriptor, get the Schema as described in Schema.fbs::Schema // This is used when a consumer needs the Schema of flight stream. Similar to // GetFlightInfo this interface may generate a new flight that was not previously // available in ListFlights. GetSchema(context.Context, *FlightDescriptor) (*SchemaResult, error) + // // Retrieve a single stream associated with a particular descriptor // associated with the referenced ticket. A Flight can be composed of one or // more streams where each stream can be retrieved using a separate opaque // ticket that the flight service uses for managing a collection of streams. DoGet(*Ticket, FlightService_DoGetServer) error + // // Push a stream to the flight service associated with a particular // flight stream. This allows a client of a flight service to upload a stream // of data. Depending on the particular flight service, a client consumer @@ -422,12 +444,14 @@ type FlightServiceServer interface { // number. In the latter, the service might implement a 'seal' action that // can be applied to a descriptor once all streams are uploaded. DoPut(FlightService_DoPutServer) error + // // Open a bidirectional data channel for a given descriptor. This // allows clients to send and receive arbitrary Arrow data and // application-specific metadata in a single logical stream. In // contrast to DoGet/DoPut, this is more suited for clients // offloading computation (rather than storage) to a Flight service. DoExchange(FlightService_DoExchangeServer) error + // // Flight services can support an arbitrary number of simple actions in // addition to the possible ListFlights, GetFlightInfo, DoGet, DoPut // operations that are potentially available. DoAction allows a flight client @@ -435,6 +459,7 @@ type FlightServiceServer interface { // opaque request and response objects that are specific to the type action // being undertaken. DoAction(*Action, FlightService_DoActionServer) error + // // A flight service exposes all of the available action types that it has // along with descriptions. This allows different flight consumers to // understand the capabilities of the flight service. @@ -485,8 +510,8 @@ type UnsafeFlightServiceServer interface { mustEmbedUnimplementedFlightServiceServer() } -func RegisterFlightServiceServer(s *grpc.Server, srv FlightServiceServer) { - s.RegisterService(&_FlightService_serviceDesc, srv) +func RegisterFlightServiceServer(s grpc.ServiceRegistrar, srv FlightServiceServer) { + s.RegisterService(&FlightService_ServiceDesc, srv) } func _FlightService_Handshake_Handler(srv interface{}, stream grpc.ServerStream) error { @@ -705,7 +730,10 @@ func (x *flightServiceListActionsServer) Send(m *ActionType) error { return x.ServerStream.SendMsg(m) } -var _FlightService_serviceDesc = grpc.ServiceDesc{ +// FlightService_ServiceDesc is the grpc.ServiceDesc for FlightService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var FlightService_ServiceDesc = grpc.ServiceDesc{ ServiceName: "arrow.flight.protocol.FlightService", HandlerType: (*FlightServiceServer)(nil), Methods: []grpc.MethodDesc{ diff --git a/go/arrow/internal/flight_integration/scenario.go b/go/arrow/internal/flight_integration/scenario.go index 4108cf8124245..4f47d7fd506ee 100644 --- a/go/arrow/internal/flight_integration/scenario.go +++ b/go/arrow/internal/flight_integration/scenario.go @@ -71,6 +71,8 @@ func GetScenario(name string, args ...string) Scenario { return &expirationTimeRenewFlightEndpointScenarioTester{} case "poll_flight_info": return &pollFlightInfoScenarioTester{} + case "app_metadata_flight_info_endpoint": + return &appMetadataFlightInfoEndpointScenarioTester{} case "flight_sql": return &flightSqlScenarioTester{} case "flight_sql:extension": @@ -1153,7 +1155,7 @@ func (tester *pollFlightInfoScenarioTester) PollFlightInfo(ctx context.Context, nil, ) endpoints := []*flight.FlightEndpoint{ - &flight.FlightEndpoint{ + { Ticket: &flight.Ticket{Ticket: []byte("long-running query")}, Location: []*flight.Location{}, }, @@ -1236,6 +1238,66 @@ func (tester *pollFlightInfoScenarioTester) RunClient(addr string, opts ...grpc. return nil } +type appMetadataFlightInfoEndpointScenarioTester struct { + flight.BaseFlightServer +} + +func (tester *appMetadataFlightInfoEndpointScenarioTester) MakeServer(port int) flight.Server { + srv := flight.NewServerWithMiddleware(nil) + srv.RegisterFlightService(tester) + initServer(port, srv) + return srv +} + +func (tester *appMetadataFlightInfoEndpointScenarioTester) GetFlightInfo(ctx context.Context, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) { + schema := arrow.NewSchema( + []arrow.Field{ + {Name: "number", Type: arrow.PrimitiveTypes.Uint32}, + }, + nil, + ) + + if desc.Type != flight.DescriptorCMD { + return nil, fmt.Errorf("%w: should have received CMD descriptor", arrow.ErrInvalid) + } + endpoints := []*flight.FlightEndpoint{{AppMetadata: desc.Cmd}} + return &flight.FlightInfo{ + Schema: flight.SerializeSchema(schema, memory.DefaultAllocator), + FlightDescriptor: desc, + Endpoint: endpoints, + TotalRecords: -1, + TotalBytes: -1, + AppMetadata: desc.Cmd, + }, nil +} + +func (tester *appMetadataFlightInfoEndpointScenarioTester) RunClient(addr string, opts ...grpc.DialOption) error { + client, err := flight.NewClientWithMiddleware(addr, nil, nil, opts...) + if err != nil { + return err + } + defer client.Close() + + ctx := context.Background() + desc := flight.FlightDescriptor{ + Type: flight.DescriptorCMD, + Cmd: []byte("foobar"), + } + info, err := client.GetFlightInfo(ctx, &desc) + if err != nil { + return err + } + switch { + case !bytes.Equal(desc.Cmd, info.AppMetadata): + return fmt.Errorf("invalid flight info app_metadata: %s, expected: %s", info.AppMetadata, desc.Cmd) + case len(info.Endpoint) != 1: + return fmt.Errorf("expected exactly 1 flight endpoint, got: %d", len(info.Endpoint)) + case !bytes.Equal(desc.Cmd, info.Endpoint[0].AppMetadata): + return fmt.Errorf("invalid flight endpoint app_metadata: %s, expected: %s", info.Endpoint[0].AppMetadata, desc.Cmd) + } + return nil +} + const ( updateStatementExpectedRows int64 = 10000 updateStatementWithTransactionExpectedRows int64 = 15000