From 003cced0513abfdd59b29f69e48ff98a6b37f766 Mon Sep 17 00:00:00 2001 From: Sutou Kouhei <kou@clear-code.com> Date: Tue, 15 Aug 2023 10:51:13 +0900 Subject: [PATCH] GH-36155: [C++][Go][Java][FlightRPC] Add support for long-running queries (#36946) ### Rationale for this change In Flight RPC, FlightInfo includes addresses of workers alongside result partition info. This lets clients fetch data directly from workers, in parallel or even distributed across multiple machines. But this also comes with tradeoffs. Queries generally don't complete instantly (as much as we would like them to). So where can we put the 'query evaluation time'? * In `GetFlightInfo`: block and wait for the query to complete. * Con: this is a long-running blocking call, which may fail or time out. Then when the client retries, the server has to redo all the work. * Con: parts of the result may be ready before others, but the client can't do anything until everything is ready. * In `DoGet`: return a fixed number of partitions * Con: this makes handling worker failures hard. Systems like Trino support fault-tolerant execution by replacing workers at runtime. But GetFlightInfo has already passed, so we can't notify the client of new workers. * Con: we have to know or fix the partitioning up front. Neither solution is optimal. ### What changes are included in this PR? We can address this by adding a retryable version of `GetFlightInfo`: `PollFlightInfo(FlightDescriptor)` `PollFlightInfo` returns `PollInfo`: ```proto message PollInfo { // The currently available results so far. FlightInfo info = 1; // The descriptor the client should use on the next try. // If unset, the query is complete. FlightDescriptor flight_descriptor = 2; // Query progress. Must be in [0.0, 1.0] but need not be // monotonic or nondecreasing. If unknown, do not set. optional double progress = 3; // 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. google.protobuf.Timestamp expiration_time = 4; } ``` See the documentation changes for details of them: http://crossbow.voltrondata.com/pr_docs/36946/format/Flight.html#downloading-data-by-running-a-heavy-query ### Are these changes tested? Yes. This has C++, Go and Java implementations and an integration test with them. ### Are there any user-facing changes? Yes. * Closes: #36155 Lead-authored-by: Sutou Kouhei <kou@clear-code.com> Co-authored-by: Sutou Kouhei <kou@cozmixng.org> Co-authored-by: David Li <li.davidm96@gmail.com> Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org> Signed-off-by: Sutou Kouhei <kou@clear-code.com> --- cpp/src/arrow/flight/CMakeLists.txt | 10 +- cpp/src/arrow/flight/client.cc | 8 + cpp/src/arrow/flight/client.h | 13 + cpp/src/arrow/flight/flight_internals_test.cc | 37 +- .../flight_integration_test.cc | 2 + .../integration_tests/test_integration.cc | 78 ++- cpp/src/arrow/flight/middleware.h | 1 + .../arrow/flight/serialization_internal.cc | 80 ++- cpp/src/arrow/flight/serialization_internal.h | 4 + cpp/src/arrow/flight/server.cc | 6 + cpp/src/arrow/flight/server.h | 14 +- cpp/src/arrow/flight/transport.cc | 5 + cpp/src/arrow/flight/transport.h | 3 + .../flight/transport/grpc/grpc_client.cc | 21 + .../flight/transport/grpc/grpc_server.cc | 26 + .../arrow/flight/transport/ucx/ucx_client.cc | 32 + .../arrow/flight/transport/ucx/ucx_internal.h | 1 + .../arrow/flight/transport/ucx/ucx_server.cc | 23 + cpp/src/arrow/flight/type_fwd.h | 1 + cpp/src/arrow/flight/types.cc | 86 +++ cpp/src/arrow/flight/types.h | 65 ++ dev/archery/archery/integration/runner.py | 5 + docs/source/format/Flight.rst | 64 ++ docs/source/format/Flight/PollFlightInfo.mmd | 38 ++ .../format/Flight/PollFlightInfo.mmd.svg | 1 + format/Flight.proto | 71 ++- go/arrow/flight/flightsql/client_test.go | 5 + go/arrow/flight/gen/flight/Flight.pb.go | 512 ++++++++++------ go/arrow/flight/gen/flight/FlightSql.pb.go | 580 ++++++++---------- go/arrow/flight/gen/flight/Flight_grpc.pb.go | 122 +++- go/arrow/flight/server.go | 1 + .../internal/flight_integration/scenario.go | 105 ++++ .../org/apache/arrow/flight/FlightClient.java | 17 + .../org/apache/arrow/flight/FlightMethod.java | 3 + .../apache/arrow/flight/FlightProducer.java | 16 + .../apache/arrow/flight/FlightService.java | 15 + .../org/apache/arrow/flight/PollInfo.java | 160 +++++ .../tests/PollFlightInfoProducer.java | 56 ++ .../tests/PollFlightInfoScenario.java | 62 ++ .../flight/integration/tests/Scenarios.java | 1 + .../integration/tests/IntegrationTest.java | 5 + 41 files changed, 1762 insertions(+), 593 deletions(-) create mode 100644 docs/source/format/Flight/PollFlightInfo.mmd create mode 100644 docs/source/format/Flight/PollFlightInfo.mmd.svg create mode 100644 java/flight/flight-core/src/main/java/org/apache/arrow/flight/PollInfo.java create mode 100644 java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/PollFlightInfoProducer.java create mode 100644 java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/PollFlightInfoScenario.java diff --git a/cpp/src/arrow/flight/CMakeLists.txt b/cpp/src/arrow/flight/CMakeLists.txt index 6e76181533459..c37d2c56700e0 100644 --- a/cpp/src/arrow/flight/CMakeLists.txt +++ b/cpp/src/arrow/flight/CMakeLists.txt @@ -83,11 +83,15 @@ set(FLIGHT_GENERATED_PROTO_FILES set(PROTO_DEPENDS ${FLIGHT_PROTO} ${ARROW_PROTOBUF_LIBPROTOBUF} gRPC::grpc_cpp_plugin) +set(FLIGHT_PROTOC_COMMAND ${ARROW_PROTOBUF_PROTOC} "-I${FLIGHT_PROTO_PATH}") +if(Protobuf_VERSION VERSION_LESS 3.15) + list(APPEND FLIGHT_PROTOC_COMMAND "--experimental_allow_proto3_optional") +endif() add_custom_command(OUTPUT ${FLIGHT_GENERATED_PROTO_FILES} - COMMAND ${ARROW_PROTOBUF_PROTOC} "-I${FLIGHT_PROTO_PATH}" - "--cpp_out=${CMAKE_CURRENT_BINARY_DIR}" "${FLIGHT_PROTO}" DEPENDS ${PROTO_DEPENDS} ARGS - COMMAND ${ARROW_PROTOBUF_PROTOC} "-I${FLIGHT_PROTO_PATH}" + COMMAND ${FLIGHT_PROTOC_COMMAND} + "--cpp_out=${CMAKE_CURRENT_BINARY_DIR}" "${FLIGHT_PROTO}" + COMMAND ${FLIGHT_PROTOC_COMMAND} "--grpc_out=${CMAKE_CURRENT_BINARY_DIR}" "--plugin=protoc-gen-grpc=$<TARGET_FILE:gRPC::grpc_cpp_plugin>" "${FLIGHT_PROTO}") diff --git a/cpp/src/arrow/flight/client.cc b/cpp/src/arrow/flight/client.cc index eb62ec65ffc5f..f8dd4ba549ff3 100644 --- a/cpp/src/arrow/flight/client.cc +++ b/cpp/src/arrow/flight/client.cc @@ -640,6 +640,14 @@ arrow::Future<FlightInfo> FlightClient::GetFlightInfoAsync( return future; } +arrow::Result<std::unique_ptr<PollInfo>> FlightClient::PollFlightInfo( + const FlightCallOptions& options, const FlightDescriptor& descriptor) { + std::unique_ptr<PollInfo> info; + RETURN_NOT_OK(CheckOpen()); + RETURN_NOT_OK(transport_->PollFlightInfo(options, descriptor, &info)); + return info; +} + arrow::Result<std::unique_ptr<SchemaResult>> FlightClient::GetSchema( const FlightCallOptions& options, const FlightDescriptor& descriptor) { RETURN_NOT_OK(CheckOpen()); diff --git a/cpp/src/arrow/flight/client.h b/cpp/src/arrow/flight/client.h index cc1c35aaebed3..705b36c23cebe 100644 --- a/cpp/src/arrow/flight/client.h +++ b/cpp/src/arrow/flight/client.h @@ -296,6 +296,19 @@ class ARROW_FLIGHT_EXPORT FlightClient { return GetFlightInfoAsync({}, descriptor); } + /// \brief Request and poll a long running query + /// \param[in] options Per-RPC options + /// \param[in] descriptor the dataset request or a descriptor returned by a + /// prioir PollFlightInfo call + /// \return Arrow result with the PollInfo describing the status of + /// the requested query + arrow::Result<std::unique_ptr<PollInfo>> PollFlightInfo( + const FlightCallOptions& options, const FlightDescriptor& descriptor); + arrow::Result<std::unique_ptr<PollInfo>> PollFlightInfo( + const FlightDescriptor& descriptor) { + return PollFlightInfo({}, descriptor); + } + /// \brief Request schema for a single flight, which may be an existing /// dataset or a command to be executed /// \param[in] options Per-RPC options diff --git a/cpp/src/arrow/flight/flight_internals_test.cc b/cpp/src/arrow/flight/flight_internals_test.cc index 72a25018e8ee8..b32036a2c973f 100644 --- a/cpp/src/arrow/flight/flight_internals_test.cc +++ b/cpp/src/arrow/flight/flight_internals_test.cc @@ -64,9 +64,12 @@ void TestRoundtrip(const std::vector<FlightType>& values, ASSERT_OK_AND_ASSIGN(std::string serialized, values[i].SerializeToString()); ASSERT_OK_AND_ASSIGN(auto deserialized, FlightType::Deserialize(serialized)); - if constexpr (std::is_same_v<FlightType, FlightInfo>) { + if constexpr (std::is_same_v<FlightType, FlightInfo> || + std::is_same_v<FlightType, PollInfo>) { + ARROW_SCOPED_TRACE("Deserialized = ", deserialized->ToString()); EXPECT_EQ(values[i], *deserialized); } else { + ARROW_SCOPED_TRACE("Deserialized = ", deserialized.ToString()); EXPECT_EQ(values[i], deserialized); } @@ -255,6 +258,38 @@ TEST(FlightTypes, FlightInfo) { ASSERT_NO_FATAL_FAILURE(TestRoundtrip<pb::FlightInfo>(values, reprs)); } +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); + // 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 + // resolution on some platforms such as Windows. + const auto expiration_time_duration = + std::chrono::seconds{1687144446} + std::chrono::nanoseconds{4339000}; + Timestamp expiration_time( + std::chrono::duration_cast<Timestamp::duration>(expiration_time_duration)); + std::vector<PollInfo> values = { + PollInfo{std::make_unique<FlightInfo>(info), std::nullopt, std::nullopt, + std::nullopt}, + PollInfo{std::make_unique<FlightInfo>(info), FlightDescriptor::Command("poll"), 0.1, + expiration_time}, + }; + std::vector<std::string> reprs = { + "<PollInfo info=" + info.ToString() + + " descriptor=null " + "progress=null expiration_time=null>", + "<PollInfo info=" + info.ToString() + + " descriptor=<FlightDescriptor cmd='poll'> " + "progress=0.1 expiration_time=2023-06-19 03:14:06.004339000>", + }; + + ASSERT_NO_FATAL_FAILURE(TestRoundtrip<pb::PollInfo>(values, reprs)); +} + TEST(FlightTypes, Result) { std::vector<Result> values = { {Buffer::FromString("")}, 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 d2f14fb01b0a8..4a49dea31b99d 100644 --- a/cpp/src/arrow/flight/integration_tests/flight_integration_test.cc +++ b/cpp/src/arrow/flight/integration_tests/flight_integration_test.cc @@ -71,6 +71,8 @@ TEST(FlightIntegration, ExpirationTimeRenewFlightEndpoint) { ASSERT_OK(RunScenario("expiration_time:renew_flight_endpoint")); } +TEST(FlightIntegration, PollFlightInfo) { ASSERT_OK(RunScenario("poll_flight_info")); } + 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 e1d39ae5d7264..03e352ffa6770 100644 --- a/cpp/src/arrow/flight/integration_tests/test_integration.cc +++ b/cpp/src/arrow/flight/integration_tests/test_integration.cc @@ -708,9 +708,7 @@ class ExpirationTimeCancelFlightInfoScenario : public Scenario { /// \brief The expiration time scenario - RenewFlightEndpoint. /// -/// This tests that the client can renew a FlightEndpoint and read -/// data in renewed expiration time even when the original -/// expiration time is over. +/// This tests that the client can renew a FlightEndpoint. class ExpirationTimeRenewFlightEndpointScenario : public Scenario { Status MakeServer(std::unique_ptr<FlightServerBase>* server, FlightServerOptions* options) override { @@ -746,6 +744,77 @@ class ExpirationTimeRenewFlightEndpointScenario : public Scenario { } }; +/// \brief The server used for testing PollFlightInfo(). +class PollFlightInfoServer : public FlightServerBase { + public: + PollFlightInfoServer() : FlightServerBase() {} + + Status PollFlightInfo(const ServerCallContext& context, + const FlightDescriptor& descriptor, + std::unique_ptr<PollInfo>* result) override { + auto schema = arrow::schema({arrow::field("number", arrow::uint32(), false)}); + std::vector<FlightEndpoint> endpoints = { + FlightEndpoint{{"long-running query"}, {}, std::nullopt}}; + ARROW_ASSIGN_OR_RAISE( + auto info, FlightInfo::Make(*schema, descriptor, endpoints, -1, -1, false)); + if (descriptor == FlightDescriptor::Command("poll")) { + *result = std::make_unique<PollInfo>(std::make_unique<FlightInfo>(std::move(info)), + std::nullopt, 1.0, std::nullopt); + } else { + *result = + std::make_unique<PollInfo>(std::make_unique<FlightInfo>(std::move(info)), + FlightDescriptor::Command("poll"), 0.1, + Timestamp::clock::now() + std::chrono::seconds{10}); + } + return Status::OK(); + } +}; + +/// \brief The PollFlightInfo scenario. +/// +/// This tests that the client can poll a long-running query. +class PollFlightInfoScenario : public Scenario { + Status MakeServer(std::unique_ptr<FlightServerBase>* server, + FlightServerOptions* options) override { + *server = std::make_unique<PollFlightInfoServer>(); + return Status::OK(); + } + + Status MakeClient(FlightClientOptions* options) override { return Status::OK(); } + + Status RunClient(std::unique_ptr<FlightClient> client) override { + ARROW_ASSIGN_OR_RAISE( + auto info, client->PollFlightInfo(FlightDescriptor::Command("heavy query"))); + if (!info->descriptor.has_value()) { + return Status::Invalid("Description is missing: ", info->ToString()); + } + if (!info->progress.has_value()) { + return Status::Invalid("Progress is missing: ", info->ToString()); + } + if (!(0.0 <= *info->progress && *info->progress <= 1.0)) { + return Status::Invalid("Invalid progress: ", info->ToString()); + } + if (!info->expiration_time.has_value()) { + return Status::Invalid("Expiration time is missing: ", info->ToString()); + } + ARROW_ASSIGN_OR_RAISE(info, client->PollFlightInfo(*info->descriptor)); + if (info->descriptor.has_value()) { + return Status::Invalid("Retried but not finished yet: ", info->ToString()); + } + if (!info->progress.has_value()) { + return Status::Invalid("Progress is missing in finished query: ", info->ToString()); + } + if (fabs(*info->progress - 1.0) > arrow::kDefaultAbsoluteTolerance) { + return Status::Invalid("Progress for finished query isn't 1.0: ", info->ToString()); + } + if (info->expiration_time.has_value()) { + return Status::Invalid("Expiration time must not be set for finished query: ", + info->ToString()); + } + return Status::OK(); + } +}; + /// \brief Schema to be returned for mocking the statement/prepared statement results. /// /// Must be the same across all languages. @@ -1825,6 +1894,9 @@ Status GetScenario(const std::string& scenario_name, std::shared_ptr<Scenario>* } else if (scenario_name == "expiration_time:renew_flight_endpoint") { *out = std::make_shared<ExpirationTimeRenewFlightEndpointScenario>(); return Status::OK(); + } else if (scenario_name == "poll_flight_info") { + *out = std::make_shared<PollFlightInfoScenario>(); + return Status::OK(); } else if (scenario_name == "flight_sql") { *out = std::make_shared<FlightSqlScenario>(); return Status::OK(); diff --git a/cpp/src/arrow/flight/middleware.h b/cpp/src/arrow/flight/middleware.h index e936b9f02020c..84448097ff019 100644 --- a/cpp/src/arrow/flight/middleware.h +++ b/cpp/src/arrow/flight/middleware.h @@ -57,6 +57,7 @@ enum class FlightMethod : char { DoAction = 7, ListActions = 8, DoExchange = 9, + PollFlightInfo = 10, }; /// \brief Get a human-readable name for a Flight method. diff --git a/cpp/src/arrow/flight/serialization_internal.cc b/cpp/src/arrow/flight/serialization_internal.cc index 5d09a1a04586c..f85b451dccec3 100644 --- a/cpp/src/arrow/flight/serialization_internal.cc +++ b/cpp/src/arrow/flight/serialization_internal.cc @@ -31,6 +31,26 @@ namespace arrow { namespace flight { namespace internal { +// Timestamp + +Status FromProto(const google::protobuf::Timestamp& pb_timestamp, Timestamp* timestamp) { + const auto seconds = std::chrono::seconds{pb_timestamp.seconds()}; + const auto nanoseconds = std::chrono::nanoseconds{pb_timestamp.nanos()}; + const auto duration = + std::chrono::duration_cast<Timestamp::duration>(seconds + nanoseconds); + *timestamp = Timestamp(duration); + return Status::OK(); +} + +Status ToProto(const Timestamp& timestamp, google::protobuf::Timestamp* pb_timestamp) { + const auto since_epoch = timestamp.time_since_epoch(); + const auto since_epoch_ns = + std::chrono::duration_cast<std::chrono::nanoseconds>(since_epoch).count(); + pb_timestamp->set_seconds(since_epoch_ns / std::nano::den); + pb_timestamp->set_nanos(since_epoch_ns % std::nano::den); + return Status::OK(); +} + // ActionType Status FromProto(const pb::ActionType& pb_type, ActionType* type) { @@ -153,13 +173,9 @@ Status FromProto(const pb::FlightEndpoint& pb_endpoint, FlightEndpoint* endpoint RETURN_NOT_OK(FromProto(pb_endpoint.location(i), &endpoint->locations[i])); } if (pb_endpoint.has_expiration_time()) { - const auto& pb_expiration_time = pb_endpoint.expiration_time(); - const auto seconds = std::chrono::seconds{pb_expiration_time.seconds()}; - const auto nanoseconds = std::chrono::nanoseconds{pb_expiration_time.nanos()}; - const auto duration = - std::chrono::duration_cast<Timestamp::duration>(seconds + nanoseconds); - const Timestamp expiration_time(duration); - endpoint->expiration_time = expiration_time; + Timestamp expiration_time; + RETURN_NOT_OK(FromProto(pb_endpoint.expiration_time(), &expiration_time)); + endpoint->expiration_time = std::move(expiration_time); } return Status::OK(); } @@ -171,13 +187,8 @@ Status ToProto(const FlightEndpoint& endpoint, pb::FlightEndpoint* pb_endpoint) RETURN_NOT_OK(ToProto(location, pb_endpoint->add_location())); } if (endpoint.expiration_time) { - const auto expiration_time = endpoint.expiration_time.value(); - const auto since_epoch = expiration_time.time_since_epoch(); - const auto since_epoch_ns = - std::chrono::duration_cast<std::chrono::nanoseconds>(since_epoch).count(); - auto pb_expiration_time = pb_endpoint->mutable_expiration_time(); - pb_expiration_time->set_seconds(since_epoch_ns / std::nano::den); - pb_expiration_time->set_nanos(since_epoch_ns % std::nano::den); + RETURN_NOT_OK(ToProto(endpoint.expiration_time.value(), + pb_endpoint->mutable_expiration_time())); } return Status::OK(); } @@ -288,6 +299,47 @@ Status ToProto(const FlightInfo& info, pb::FlightInfo* pb_info) { return Status::OK(); } +// PollInfo + +Status FromProto(const pb::PollInfo& pb_info, PollInfo* info) { + ARROW_ASSIGN_OR_RAISE(auto flight_info, FromProto(pb_info.info())); + info->info = std::make_unique<FlightInfo>(std::move(flight_info)); + if (pb_info.has_flight_descriptor()) { + FlightDescriptor descriptor; + RETURN_NOT_OK(FromProto(pb_info.flight_descriptor(), &descriptor)); + info->descriptor = std::move(descriptor); + } else { + info->descriptor = std::nullopt; + } + if (pb_info.has_progress()) { + info->progress = pb_info.progress(); + } else { + info->progress = std::nullopt; + } + if (pb_info.has_expiration_time()) { + Timestamp expiration_time; + RETURN_NOT_OK(FromProto(pb_info.expiration_time(), &expiration_time)); + info->expiration_time = std::move(expiration_time); + } else { + info->expiration_time = std::nullopt; + } + return Status::OK(); +} + +Status ToProto(const PollInfo& info, pb::PollInfo* pb_info) { + RETURN_NOT_OK(ToProto(*info.info, pb_info->mutable_info())); + if (info.descriptor) { + RETURN_NOT_OK(ToProto(*info.descriptor, pb_info->mutable_flight_descriptor())); + } + if (info.progress) { + pb_info->set_progress(info.progress.value()); + } + if (info.expiration_time) { + RETURN_NOT_OK(ToProto(*info.expiration_time, pb_info->mutable_expiration_time())); + } + return Status::OK(); +} + // CancelFlightInfoRequest Status FromProto(const pb::CancelFlightInfoRequest& pb_request, diff --git a/cpp/src/arrow/flight/serialization_internal.h b/cpp/src/arrow/flight/serialization_internal.h index 30eb0b31819b8..1ac7de83d1308 100644 --- a/cpp/src/arrow/flight/serialization_internal.h +++ b/cpp/src/arrow/flight/serialization_internal.h @@ -45,6 +45,7 @@ Status SchemaToString(const Schema& schema, std::string* out); // These functions depend on protobuf types which are not exported in the Flight DLL. +Status FromProto(const google::protobuf::Timestamp& pb_timestamp, Timestamp* timestamp); Status FromProto(const pb::ActionType& pb_type, ActionType* type); Status FromProto(const pb::Action& pb_action, Action* action); Status FromProto(const pb::Result& pb_result, Result* result); @@ -60,16 +61,19 @@ Status FromProto(const pb::FlightEndpoint& pb_endpoint, FlightEndpoint* endpoint Status FromProto(const pb::RenewFlightEndpointRequest& pb_request, RenewFlightEndpointRequest* request); arrow::Result<FlightInfo> FromProto(const pb::FlightInfo& pb_info); +Status FromProto(const pb::PollInfo& pb_info, PollInfo* info); Status FromProto(const pb::CancelFlightInfoRequest& pb_request, CancelFlightInfoRequest* request); Status FromProto(const pb::SchemaResult& pb_result, std::string* result); Status FromProto(const pb::BasicAuth& pb_basic_auth, BasicAuth* info); +Status ToProto(const Timestamp& timestamp, google::protobuf::Timestamp* pb_timestamp); Status ToProto(const FlightDescriptor& descr, pb::FlightDescriptor* pb_descr); Status ToProto(const FlightEndpoint& endpoint, pb::FlightEndpoint* pb_endpoint); Status ToProto(const RenewFlightEndpointRequest& request, pb::RenewFlightEndpointRequest* pb_request); Status ToProto(const FlightInfo& info, pb::FlightInfo* pb_info); +Status ToProto(const PollInfo& info, pb::PollInfo* pb_info); Status ToProto(const CancelFlightInfoRequest& request, pb::CancelFlightInfoRequest* pb_request); Status ToProto(const ActionType& type, pb::ActionType* pb_type); diff --git a/cpp/src/arrow/flight/server.cc b/cpp/src/arrow/flight/server.cc index 66185cfeba5e5..06512bda36a49 100644 --- a/cpp/src/arrow/flight/server.cc +++ b/cpp/src/arrow/flight/server.cc @@ -231,6 +231,12 @@ Status FlightServerBase::GetFlightInfo(const ServerCallContext& context, return Status::NotImplemented("NYI"); } +Status FlightServerBase::PollFlightInfo(const ServerCallContext& context, + const FlightDescriptor& request, + std::unique_ptr<PollInfo>* info) { + return Status::NotImplemented("NYI"); +} + Status FlightServerBase::DoGet(const ServerCallContext& context, const Ticket& request, std::unique_ptr<FlightDataStream>* data_stream) { return Status::NotImplemented("NYI"); diff --git a/cpp/src/arrow/flight/server.h b/cpp/src/arrow/flight/server.h index 049c6cee3ffcf..6eba90c53a754 100644 --- a/cpp/src/arrow/flight/server.h +++ b/cpp/src/arrow/flight/server.h @@ -252,16 +252,26 @@ class ARROW_FLIGHT_EXPORT FlightServerBase { /// \brief Retrieve the schema and an access plan for the indicated /// descriptor /// \param[in] context The call context. - /// \param[in] request may be null + /// \param[in] request the dataset request, whether a named dataset or command /// \param[out] info the returned flight info provider /// \return Status virtual Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, std::unique_ptr<FlightInfo>* info); + /// \brief Retrieve the current status of the target query + /// \param[in] context The call context. + /// \param[in] request the dataset request or a descriptor returned by a + /// prioir PollFlightInfo call + /// \param[out] info the returned retry info provider + /// \return Status + virtual Status PollFlightInfo(const ServerCallContext& context, + const FlightDescriptor& request, + std::unique_ptr<PollInfo>* info); + /// \brief Retrieve the schema for the indicated descriptor /// \param[in] context The call context. - /// \param[in] request may be null + /// \param[in] request the dataset request, whether a named dataset or command /// \param[out] schema the returned flight schema provider /// \return Status virtual Status GetSchema(const ServerCallContext& context, diff --git a/cpp/src/arrow/flight/transport.cc b/cpp/src/arrow/flight/transport.cc index 88228f2503597..fd74b6d95a59e 100644 --- a/cpp/src/arrow/flight/transport.cc +++ b/cpp/src/arrow/flight/transport.cc @@ -80,6 +80,11 @@ void ClientTransport::GetFlightInfoAsync( std::shared_ptr<AsyncListener<FlightInfo>> listener) { listener->OnFinish(Status::NotImplemented("Async GetFlightInfo for this transport")); } +Status ClientTransport::PollFlightInfo(const FlightCallOptions& options, + const FlightDescriptor& descriptor, + std::unique_ptr<PollInfo>* info) { + return Status::NotImplemented("PollFlightInfo for this transport"); +} arrow::Result<std::unique_ptr<SchemaResult>> ClientTransport::GetSchema( const FlightCallOptions& options, const FlightDescriptor& descriptor) { return Status::NotImplemented("GetSchema for this transport"); diff --git a/cpp/src/arrow/flight/transport.h b/cpp/src/arrow/flight/transport.h index 69605d2112f1f..8cb2479d113fc 100644 --- a/cpp/src/arrow/flight/transport.h +++ b/cpp/src/arrow/flight/transport.h @@ -187,6 +187,9 @@ class ARROW_FLIGHT_EXPORT ClientTransport { virtual void GetFlightInfoAsync(const FlightCallOptions& options, const FlightDescriptor& descriptor, std::shared_ptr<AsyncListener<FlightInfo>> listener); + virtual Status PollFlightInfo(const FlightCallOptions& options, + const FlightDescriptor& descriptor, + std::unique_ptr<PollInfo>* info); virtual arrow::Result<std::unique_ptr<SchemaResult>> GetSchema( const FlightCallOptions& options, const FlightDescriptor& descriptor); virtual Status ListFlights(const FlightCallOptions& options, const Criteria& criteria, diff --git a/cpp/src/arrow/flight/transport/grpc/grpc_client.cc b/cpp/src/arrow/flight/transport/grpc/grpc_client.cc index 7108f35549998..5baf687cd83dc 100644 --- a/cpp/src/arrow/flight/transport/grpc/grpc_client.cc +++ b/cpp/src/arrow/flight/transport/grpc/grpc_client.cc @@ -179,6 +179,8 @@ class GrpcClientInterceptorAdapterFactory flight_method = FlightMethod::ListFlights; } else if (EndsWith(method, "/GetFlightInfo")) { flight_method = FlightMethod::GetFlightInfo; + } else if (EndsWith(method, "/PollFlightInfo")) { + flight_method = FlightMethod::PollFlightInfo; } else if (EndsWith(method, "/GetSchema")) { flight_method = FlightMethod::GetSchema; } else if (EndsWith(method, "/DoGet")) { @@ -942,6 +944,25 @@ class GrpcClientImpl : public internal::ClientTransport { return Status::OK(); } + Status PollFlightInfo(const FlightCallOptions& options, + const FlightDescriptor& descriptor, + std::unique_ptr<PollInfo>* info) override { + pb::FlightDescriptor pb_descriptor; + pb::PollInfo pb_response; + + RETURN_NOT_OK(internal::ToProto(descriptor, &pb_descriptor)); + + ClientRpc rpc(options); + RETURN_NOT_OK(rpc.SetToken(auth_handler_.get())); + Status s = FromGrpcStatus( + stub_->PollFlightInfo(&rpc.context, pb_descriptor, &pb_response), &rpc.context); + RETURN_NOT_OK(s); + + info->reset(new PollInfo()); + RETURN_NOT_OK(internal::FromProto(pb_response, info->get())); + return Status::OK(); + } + arrow::Result<std::unique_ptr<SchemaResult>> GetSchema( const FlightCallOptions& options, const FlightDescriptor& descriptor) override { pb::FlightDescriptor pb_descriptor; diff --git a/cpp/src/arrow/flight/transport/grpc/grpc_server.cc b/cpp/src/arrow/flight/transport/grpc/grpc_server.cc index 50d4ffe002c7e..28c2090c95a9a 100644 --- a/cpp/src/arrow/flight/transport/grpc/grpc_server.cc +++ b/cpp/src/arrow/flight/transport/grpc/grpc_server.cc @@ -412,6 +412,32 @@ class GrpcServiceHandler final : public FlightService::Service { RETURN_WITH_MIDDLEWARE(flight_context, ::grpc::Status::OK); } + ::grpc::Status PollFlightInfo(ServerContext* context, + const pb::FlightDescriptor* request, + pb::PollInfo* response) { + GrpcServerCallContext flight_context(context); + GRPC_RETURN_NOT_GRPC_OK( + CheckAuth(FlightMethod::PollFlightInfo, context, flight_context)); + + CHECK_ARG_NOT_NULL(flight_context, request, "FlightDescriptor cannot be null"); + + FlightDescriptor descr; + SERVICE_RETURN_NOT_OK(flight_context, internal::FromProto(*request, &descr)); + + std::unique_ptr<PollInfo> info; + SERVICE_RETURN_NOT_OK(flight_context, + impl_->base()->PollFlightInfo(flight_context, descr, &info)); + + if (!info) { + // Treat null listing as no flights available + RETURN_WITH_MIDDLEWARE(flight_context, ::grpc::Status(::grpc::StatusCode::NOT_FOUND, + "Flight not found")); + } + + SERVICE_RETURN_NOT_OK(flight_context, internal::ToProto(*info, response)); + RETURN_WITH_MIDDLEWARE(flight_context, ::grpc::Status::OK); + } + ::grpc::Status GetSchema(ServerContext* context, const pb::FlightDescriptor* request, pb::SchemaResult* response) { GrpcServerCallContext flight_context(context); diff --git a/cpp/src/arrow/flight/transport/ucx/ucx_client.cc b/cpp/src/arrow/flight/transport/ucx/ucx_client.cc index d11adb54adf93..cd9ddaa85a6f6 100644 --- a/cpp/src/arrow/flight/transport/ucx/ucx_client.cc +++ b/cpp/src/arrow/flight/transport/ucx/ucx_client.cc @@ -600,6 +600,38 @@ class UcxClientImpl : public arrow::flight::internal::ClientTransport { return MergeStatuses(std::move(status), ReturnConnection(std::move(connection))); } + Status PollFlightInfo(const FlightCallOptions& options, + const FlightDescriptor& descriptor, + std::unique_ptr<PollInfo>* info) override { + ARROW_ASSIGN_OR_RAISE(auto connection, CheckoutConnection(options)); + UcpCallDriver* driver = connection.driver(); + + auto impl = [&]() { + RETURN_NOT_OK(driver->StartCall(kMethodPollFlightInfo)); + + ARROW_ASSIGN_OR_RAISE(std::string payload, descriptor.SerializeToString()); + + RETURN_NOT_OK(driver->SendFrame(FrameType::kBuffer, + reinterpret_cast<const uint8_t*>(payload.data()), + static_cast<int64_t>(payload.size()))); + + ARROW_ASSIGN_OR_RAISE(auto incoming_message, driver->ReadNextFrame()); + if (incoming_message->type == FrameType::kBuffer) { + ARROW_ASSIGN_OR_RAISE( + *info, PollInfo::Deserialize(std::string_view(*incoming_message->buffer))); + ARROW_ASSIGN_OR_RAISE(incoming_message, driver->ReadNextFrame()); + } + RETURN_NOT_OK(driver->ExpectFrameType(*incoming_message, FrameType::kHeaders)); + ARROW_ASSIGN_OR_RAISE(auto headers, + HeadersFrame::Parse(std::move(incoming_message->buffer))); + Status status; + RETURN_NOT_OK(headers.GetStatus(&status)); + return status; + }; + auto status = impl(); + return MergeStatuses(std::move(status), ReturnConnection(std::move(connection))); + } + Status DoExchange(const FlightCallOptions& options, std::unique_ptr<internal::ClientDataStream>* out) override { ARROW_ASSIGN_OR_RAISE(auto connection, CheckoutConnection(options)); diff --git a/cpp/src/arrow/flight/transport/ucx/ucx_internal.h b/cpp/src/arrow/flight/transport/ucx/ucx_internal.h index d14296db097b4..c46f81eb7498b 100644 --- a/cpp/src/arrow/flight/transport/ucx/ucx_internal.h +++ b/cpp/src/arrow/flight/transport/ucx/ucx_internal.h @@ -49,6 +49,7 @@ static constexpr char kMethodDoExchange[] = "DoExchange"; static constexpr char kMethodDoGet[] = "DoGet"; static constexpr char kMethodDoPut[] = "DoPut"; static constexpr char kMethodGetFlightInfo[] = "GetFlightInfo"; +static constexpr char kMethodPollFlightInfo[] = "PollFlightInfo"; /// The header encoding the transport status. static constexpr char kHeaderStatus[] = "flight-status"; diff --git a/cpp/src/arrow/flight/transport/ucx/ucx_server.cc b/cpp/src/arrow/flight/transport/ucx/ucx_server.cc index 8bbac34705c23..b20f8f286e3bc 100644 --- a/cpp/src/arrow/flight/transport/ucx/ucx_server.cc +++ b/cpp/src/arrow/flight/transport/ucx/ucx_server.cc @@ -385,6 +385,27 @@ class UcxServerImpl : public arrow::flight::internal::ServerTransport { return Status::OK(); } + Status HandlePollFlightInfo(UcpCallDriver* driver) { + UcxServerCallContext context; + + ARROW_ASSIGN_OR_RAISE(auto frame, driver->ReadNextFrame()); + SERVER_RETURN_NOT_OK(driver, driver->ExpectFrameType(*frame, FrameType::kBuffer)); + FlightDescriptor descriptor; + SERVER_RETURN_NOT_OK(driver, + FlightDescriptor::Deserialize(std::string_view(*frame->buffer)) + .Value(&descriptor)); + + std::unique_ptr<PollInfo> info; + std::string response; + SERVER_RETURN_NOT_OK(driver, base_->PollFlightInfo(context, descriptor, &info)); + SERVER_RETURN_NOT_OK(driver, info->SerializeToString().Value(&response)); + RETURN_NOT_OK(driver->SendFrame(FrameType::kBuffer, + reinterpret_cast<const uint8_t*>(response.data()), + static_cast<int64_t>(response.size()))); + RETURN_NOT_OK(SendStatus(driver, Status::OK())); + return Status::OK(); + } + Status HandleDoGet(UcpCallDriver* driver) { UcxServerCallContext context; @@ -431,6 +452,8 @@ class UcxServerImpl : public arrow::flight::internal::ServerTransport { ARROW_ASSIGN_OR_RAISE(auto method, headers.Get(":method:")); if (method == kMethodGetFlightInfo) { return HandleGetFlightInfo(driver); + } else if (method == kMethodPollFlightInfo) { + return HandlePollFlightInfo(driver); } else if (method == kMethodDoExchange) { return HandleDoExchange(driver); } else if (method == kMethodDoGet) { diff --git a/cpp/src/arrow/flight/type_fwd.h b/cpp/src/arrow/flight/type_fwd.h index ac2effbc91d96..2f22bbea36dbb 100644 --- a/cpp/src/arrow/flight/type_fwd.h +++ b/cpp/src/arrow/flight/type_fwd.h @@ -38,6 +38,7 @@ struct FlightClientOptions; struct FlightDescriptor; struct FlightEndpoint; class FlightInfo; +class PollInfo; class FlightListing; class FlightMetadataReader; class FlightMetadataWriter; diff --git a/cpp/src/arrow/flight/types.cc b/cpp/src/arrow/flight/types.cc index b7cd55325b1f4..8395919e4daeb 100644 --- a/cpp/src/arrow/flight/types.cc +++ b/cpp/src/arrow/flight/types.cc @@ -339,6 +339,92 @@ bool FlightInfo::Equals(const FlightInfo& other) const { data_.ordered == other.data_.ordered; } +arrow::Result<std::string> PollInfo::SerializeToString() const { + pb::PollInfo pb_info; + RETURN_NOT_OK(internal::ToProto(*this, &pb_info)); + + std::string out; + if (!pb_info.SerializeToString(&out)) { + return Status::IOError("Serialized PollInfo exceeded 2 GiB limit"); + } + return out; +} + +arrow::Result<std::unique_ptr<PollInfo>> PollInfo::Deserialize( + std::string_view serialized) { + pb::PollInfo pb_info; + if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) { + return Status::Invalid("Serialized PollInfo size should not exceed 2 GiB"); + } + google::protobuf::io::ArrayInputStream input(serialized.data(), + static_cast<int>(serialized.size())); + if (!pb_info.ParseFromZeroCopyStream(&input)) { + return Status::Invalid("Not a valid PollInfo"); + } + PollInfo info; + RETURN_NOT_OK(internal::FromProto(pb_info, &info)); + return std::make_unique<PollInfo>(std::move(info)); +} + +std::string PollInfo::ToString() const { + std::stringstream ss; + ss << "<PollInfo info=" << info->ToString(); + ss << " descriptor="; + if (descriptor) { + ss << descriptor->ToString(); + } else { + ss << "null"; + } + ss << " progress="; + if (progress) { + ss << progress.value(); + } else { + ss << "null"; + } + ss << " expiration_time="; + if (expiration_time) { + auto type = timestamp(TimeUnit::NANO); + arrow::internal::StringFormatter<TimestampType> formatter(type.get()); + auto expiration_timestamp = std::chrono::duration_cast<std::chrono::nanoseconds>( + expiration_time->time_since_epoch()) + .count(); + formatter(expiration_timestamp, + [&ss](std::string_view formatted) { ss << formatted; }); + } else { + ss << "null"; + } + ss << '>'; + return ss.str(); +} + +bool PollInfo::Equals(const PollInfo& other) const { + if ((info.get() != nullptr) != (other.info.get() != nullptr)) { + return false; + } + if (info && *info != *other.info) { + return false; + } + if (descriptor.has_value() != other.descriptor.has_value()) { + return false; + } + if (descriptor && *descriptor != *other.descriptor) { + return false; + } + if (progress.has_value() != other.progress.has_value()) { + return false; + } + if (progress && fabs(*progress - *other.progress) > arrow::kDefaultAbsoluteTolerance) { + return false; + } + if (expiration_time.has_value() != other.expiration_time.has_value()) { + return false; + } + if (expiration_time && *expiration_time != *other.expiration_time) { + return false; + } + return true; +} + std::string CancelFlightInfoRequest::ToString() const { std::stringstream ss; ss << "<CancelFlightInfoRequest info=" << info->ToString() << ">"; diff --git a/cpp/src/arrow/flight/types.h b/cpp/src/arrow/flight/types.h index c5d72d5167271..70e7c90676b73 100644 --- a/cpp/src/arrow/flight/types.h +++ b/cpp/src/arrow/flight/types.h @@ -654,6 +654,71 @@ class ARROW_FLIGHT_EXPORT FlightInfo { mutable bool reconstructed_schema_; }; +/// \brief The information to process a long-running query. +class ARROW_FLIGHT_EXPORT PollInfo { + public: + /// The currently available results so far. + std::unique_ptr<FlightInfo> info = NULLPTR; + /// The descriptor the client should use on the next try. If unset, + /// the query is complete. + std::optional<FlightDescriptor> descriptor = std::nullopt; + /// Query progress. Must be in [0.0, 1.0] but need not be + /// monotonic or nondecreasing. If unknown, do not set. + std::optional<double> progress = std::nullopt; + /// Expiration time for this request. After this passes, the server + /// might not accept the poll descriptor anymore (and the query may + /// be cancelled). This may be updated on a call to PollFlightInfo. + std::optional<Timestamp> expiration_time = std::nullopt; + + PollInfo() + : info(NULLPTR), + descriptor(std::nullopt), + progress(std::nullopt), + expiration_time(std::nullopt) {} + + explicit PollInfo(std::unique_ptr<FlightInfo> info, + std::optional<FlightDescriptor> descriptor, + std::optional<double> progress, + std::optional<Timestamp> expiration_time) + : info(std::move(info)), + descriptor(std::move(descriptor)), + progress(progress), + expiration_time(expiration_time) {} + + explicit PollInfo(const PollInfo& other) + : info(other.info ? std::make_unique<FlightInfo>(*other.info) : NULLPTR), + descriptor(other.descriptor), + progress(other.progress), + expiration_time(other.expiration_time) {} + + /// \brief Get the wire-format representation of this type. + /// + /// Useful when interoperating with non-Flight systems (e.g. REST + /// services) that may want to return Flight types. + arrow::Result<std::string> SerializeToString() const; + + /// \brief Parse the wire-format representation of this type. + /// + /// Useful when interoperating with non-Flight systems (e.g. REST + /// services) that may want to return Flight types. + static arrow::Result<std::unique_ptr<PollInfo>> Deserialize( + std::string_view serialized); + + std::string ToString() const; + + /// Compare two PollInfo for equality. This will compare the + /// serialized schema representations, NOT the logical equality of + /// the schemas. + bool Equals(const PollInfo& other) const; + + friend bool operator==(const PollInfo& left, const PollInfo& right) { + return left.Equals(right); + } + friend bool operator!=(const PollInfo& left, const PollInfo& right) { + return !(left == right); + } +}; + /// \brief The request of the CancelFlightInfoRequest action. struct ARROW_FLIGHT_EXPORT CancelFlightInfoRequest { std::unique_ptr<FlightInfo> info; diff --git a/dev/archery/archery/integration/runner.py b/dev/archery/archery/integration/runner.py index af092b92e89a8..966da84e655b2 100644 --- a/dev/archery/archery/integration/runner.py +++ b/dev/archery/archery/integration/runner.py @@ -460,6 +460,11 @@ def run_all_tests(with_cpp=True, with_java=True, with_js=True, "RenewFlightEndpoint are working as expected."), skip={"JS", "C#", "Rust"}, ), + Scenario( + "poll_flight_info", + description="Ensure PollFlightInfo is supported.", + skip={"JS", "C#", "Rust"} + ), Scenario( "flight_sql", description="Ensure Flight SQL protocol is working as expected.", diff --git a/docs/source/format/Flight.rst b/docs/source/format/Flight.rst index 97818a2e17bf7..bea1c8f5e51de 100644 --- a/docs/source/format/Flight.rst +++ b/docs/source/format/Flight.rst @@ -153,6 +153,70 @@ A client that wishes to download the data would: .. _google.protobuf.Timestamp: https://protobuf.dev/reference/protobuf/google.protobuf/#timestamp +Downloading Data by Running a Heavy Query +----------------------------------------- + +A client may need to request a heavy query to download +data. However, ``GetFlightInfo`` doesn't return until the query +completes, so the client is blocked. In this situation, the client +can use ``PollFlightInfo`` instead of ``GetFlightInfo``: + +.. figure:: ./Flight/PollFlightInfo.mmd.svg + + Polling a long-running query by ``PollFlightInfo``. + +#. Construct or acquire a ``FlightDescriptor``, as before. +#. Call ``PollFlightInfo(FlightDescriptor)`` to get a ``PollInfo`` + message. + + A server should respond as quickly as possible on the first + call. So the client shouldn't wait for the first ``PollInfo`` + response. + + If the query isn't finished, ``PollInfo.flight_descriptor`` has a + ``FlightDescriptor``. The client should use the descriptor (not the + original ``FlightDescriptor``) to call the next + ``PollFlightInfo()``. A server should recognize a + ``PollInfo.flight_descriptor`` that is not necessarily the latest + in case the client misses an update in between. + + If the query is finished, ``PollInfo.flight_descriptor`` is + unset. + + ``PollInfo.info`` is the currently available results so far. It's + a complete ``FlightInfo`` each time not just the delta between the + previous and current ``FlightInfo``. A server should only append to + the endpoints in ``PollInfo.info`` each time. So the client can + run ``DoGet(Ticket)`` with the ``Ticket`` in the ``PollInfo.info`` + even when the query isn't finished yet. ``FlightInfo.ordered`` is + also valid. + + A server should not respond until the result would be different + from last time. That way, the client can "long poll" for updates + without constantly making requests. Clients can set a short timeout + to avoid blocking calls if desired. + + ``PollInfo.progress`` may be set. It represents progress of the + query. If it's set, the value must be in ``[0.0, 1.0]``. The value + is not necessarily monotonic or nondecreasing. A server may respond by + only updating the ``PollInfo.progress`` value, though it shouldn't + spam the client with updates. + + ``PollInfo.timestamp`` is the expiration time for this + request. After this passes, a server might not accept the poll + descriptor anymore and the query may be cancelled. This may be + updated on a call to ``PollFlightInfo``. The expiration time is + represented as `google.protobuf.Timestamp`_. + + A client may be able to cancel the query by the + ``CancelFlightInfo`` action. + + A server should return an error status instead of a response if the + query fails. The client should not poll the request except for + ``TIMED_OUT`` and ``UNAVAILABLE``, which may not originate from the + server. +#. Consume each endpoint returned by the server, as before. + Uploading Data -------------- diff --git a/docs/source/format/Flight/PollFlightInfo.mmd b/docs/source/format/Flight/PollFlightInfo.mmd new file mode 100644 index 0000000000000..5a643654c0c41 --- /dev/null +++ b/docs/source/format/Flight/PollFlightInfo.mmd @@ -0,0 +1,38 @@ +%% Licensed to the Apache Software Foundation (ASF) under one +%% or more contributor license agreements. See the NOTICE file +%% distributed with this work for additional information +%% regarding copyright ownership. The ASF licenses this file +%% to you under the Apache License, Version 2.0 (the +%% "License"); you may not use this file except in compliance +%% with the License. You may obtain a copy of the License at +%% +%% http://www.apache.org/licenses/LICENSE-2.0 +%% +%% Unless required by applicable law or agreed to in writing, +%% software distributed under the License is distributed on an +%% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +%% KIND, either express or implied. See the License for the +%% specific language governing permissions and limitations +%% under the License. + +%% To generate the diagram, use mermaid-cli +%% Example: docker run --rm -v $(pwd):/data minlag/mermaid-cli -i /data/PollFlightInfo.mmd + +sequenceDiagram +autonumber + +participant Client +participant Metadata Server +participant Data Server +Client->>Metadata Server: PollFlightInfo(FlightDescriptor) +Metadata Server->>Client: RetryInfo{descriptor: FlightDescriptor', ...} +Client->>Metadata Server: PollFlightInfo(FlightDescriptor') +Metadata Server->>Client: RetryInfo{descriptor: FlightDescriptor'', ...} +Client->>Metadata Server: PollFlightInfo(FlightDescriptor'') +Metadata Server->>Client: RetryInfo{descriptor: null, info: FlightInfo{endpoints: [FlightEndpoint{ticket: Ticket}, …]} +Note over Client, Data Server: This may be parallelized +Note over Client, Data Server: Some endpoints may be processed while polling +loop for each endpoint in FlightInfo.endpoints + Client->>Data Server: DoGet(Ticket) + Data Server->>Client: stream of FlightData +end diff --git a/docs/source/format/Flight/PollFlightInfo.mmd.svg b/docs/source/format/Flight/PollFlightInfo.mmd.svg new file mode 100644 index 0000000000000..d7b62f180aa85 --- /dev/null +++ b/docs/source/format/Flight/PollFlightInfo.mmd.svg @@ -0,0 +1 @@ +<svg aria-roledescription="sequence" role="graphics-document document" viewBox="-50 -10 1125.5 688" style="max-width: 1125.5px; background-color: white;" xmlns:xlink="http://www.w3.org/1999/xlink" xmlns="http://www.w3.org/2000/svg" width="100%" id="mermaid-1690782830274"><style>#mermaid-1690782830274{font-family:"trebuchet ms",verdana,arial,sans-serif;font-size:16px;fill:#333;}#mermaid-1690782830274 .error-icon{fill:#552222;}#mermaid-1690782830274 .error-text{fill:#552222;stroke:#552222;}#mermaid-1690782830274 .edge-thickness-normal{stroke-width:2px;}#mermaid-1690782830274 .edge-thickness-thick{stroke-width:3.5px;}#mermaid-1690782830274 .edge-pattern-solid{stroke-dasharray:0;}#mermaid-1690782830274 .edge-pattern-dashed{stroke-dasharray:3;}#mermaid-1690782830274 .edge-pattern-dotted{stroke-dasharray:2;}#mermaid-1690782830274 .marker{fill:#333333;stroke:#333333;}#mermaid-1690782830274 .marker.cross{stroke:#333333;}#mermaid-1690782830274 svg{font-family:"trebuchet ms",verdana,arial,sans-serif;font-size:16px;}#mermaid-1690782830274 .actor{stroke:hsl(259.6261682243, 59.7765363128%, 87.9019607843%);fill:#ECECFF;}#mermaid-1690782830274 text.actor>tspan{fill:black;stroke:none;}#mermaid-1690782830274 .actor-line{stroke:grey;}#mermaid-1690782830274 .messageLine0{stroke-width:1.5;stroke-dasharray:none;stroke:#333;}#mermaid-1690782830274 .messageLine1{stroke-width:1.5;stroke-dasharray:2,2;stroke:#333;}#mermaid-1690782830274 #arrowhead path{fill:#333;stroke:#333;}#mermaid-1690782830274 .sequenceNumber{fill:white;}#mermaid-1690782830274 #sequencenumber{fill:#333;}#mermaid-1690782830274 #crosshead path{fill:#333;stroke:#333;}#mermaid-1690782830274 .messageText{fill:#333;stroke:none;}#mermaid-1690782830274 .labelBox{stroke:hsl(259.6261682243, 59.7765363128%, 87.9019607843%);fill:#ECECFF;}#mermaid-1690782830274 .labelText,#mermaid-1690782830274 .labelText>tspan{fill:black;stroke:none;}#mermaid-1690782830274 .loopText,#mermaid-1690782830274 .loopText>tspan{fill:black;stroke:none;}#mermaid-1690782830274 .loopLine{stroke-width:2px;stroke-dasharray:2,2;stroke:hsl(259.6261682243, 59.7765363128%, 87.9019607843%);fill:hsl(259.6261682243, 59.7765363128%, 87.9019607843%);}#mermaid-1690782830274 .note{stroke:#aaaa33;fill:#fff5ad;}#mermaid-1690782830274 .noteText,#mermaid-1690782830274 .noteText>tspan{fill:black;stroke:none;}#mermaid-1690782830274 .activation0{fill:#f4f4f4;stroke:#666;}#mermaid-1690782830274 .activation1{fill:#f4f4f4;stroke:#666;}#mermaid-1690782830274 .activation2{fill:#f4f4f4;stroke:#666;}#mermaid-1690782830274 .actorPopupMenu{position:absolute;}#mermaid-1690782830274 .actorPopupMenuPanel{position:absolute;fill:#ECECFF;box-shadow:0px 8px 16px 0px rgba(0,0,0,0.2);filter:drop-shadow(3px 5px 2px rgb(0 0 0 / 0.4));}#mermaid-1690782830274 .actor-man line{stroke:hsl(259.6261682243, 59.7765363128%, 87.9019607843%);fill:#ECECFF;}#mermaid-1690782830274 .actor-man circle,#mermaid-1690782830274 line{stroke:hsl(259.6261682243, 59.7765363128%, 87.9019607843%);fill:#ECECFF;stroke-width:2px;}#mermaid-1690782830274 :root{--mermaid-font-family:"trebuchet ms",verdana,arial,sans-serif;}</style><g/><defs><symbol height="24" width="24" id="computer"><path d="M2 2v13h20v-13h-20zm18 11h-16v-9h16v9zm-10.228 6l.466-1h3.524l.467 1h-4.457zm14.228 3h-24l2-6h2.104l-1.33 4h18.45l-1.297-4h2.073l2 6zm-5-10h-14v-7h14v7z" transform="scale(.5)"/></symbol></defs><defs><symbol clip-rule="evenodd" fill-rule="evenodd" id="database"><path d="M12.258.001l.256.004.255.005.253.008.251.01.249.012.247.015.246.016.242.019.241.02.239.023.236.024.233.027.231.028.229.031.225.032.223.034.22.036.217.038.214.04.211.041.208.043.205.045.201.046.198.048.194.05.191.051.187.053.183.054.18.056.175.057.172.059.168.06.163.061.16.063.155.064.15.066.074.033.073.033.071.034.07.034.069.035.068.035.067.035.066.035.064.036.064.036.062.036.06.036.06.037.058.037.058.037.055.038.055.038.053.038.052.038.051.039.05.039.048.039.047.039.045.04.044.04.043.04.041.04.04.041.039.041.037.041.036.041.034.041.033.042.032.042.03.042.029.042.027.042.026.043.024.043.023.043.021.043.02.043.018.044.017.043.015.044.013.044.012.044.011.045.009.044.007.045.006.045.004.045.002.045.001.045v17l-.001.045-.002.045-.004.045-.006.045-.007.045-.009.044-.011.045-.012.044-.013.044-.015.044-.017.043-.018.044-.02.043-.021.043-.023.043-.024.043-.026.043-.027.042-.029.042-.03.042-.032.042-.033.042-.034.041-.036.041-.037.041-.039.041-.04.041-.041.04-.043.04-.044.04-.045.04-.047.039-.048.039-.05.039-.051.039-.052.038-.053.038-.055.038-.055.038-.058.037-.058.037-.06.037-.06.036-.062.036-.064.036-.064.036-.066.035-.067.035-.068.035-.069.035-.07.034-.071.034-.073.033-.074.033-.15.066-.155.064-.16.063-.163.061-.168.06-.172.059-.175.057-.18.056-.183.054-.187.053-.191.051-.194.05-.198.048-.201.046-.205.045-.208.043-.211.041-.214.04-.217.038-.22.036-.223.034-.225.032-.229.031-.231.028-.233.027-.236.024-.239.023-.241.02-.242.019-.246.016-.247.015-.249.012-.251.01-.253.008-.255.005-.256.004-.258.001-.258-.001-.256-.004-.255-.005-.253-.008-.251-.01-.249-.012-.247-.015-.245-.016-.243-.019-.241-.02-.238-.023-.236-.024-.234-.027-.231-.028-.228-.031-.226-.032-.223-.034-.22-.036-.217-.038-.214-.04-.211-.041-.208-.043-.204-.045-.201-.046-.198-.048-.195-.05-.19-.051-.187-.053-.184-.054-.179-.056-.176-.057-.172-.059-.167-.06-.164-.061-.159-.063-.155-.064-.151-.066-.074-.033-.072-.033-.072-.034-.07-.034-.069-.035-.068-.035-.067-.035-.066-.035-.064-.036-.063-.036-.062-.036-.061-.036-.06-.037-.058-.037-.057-.037-.056-.038-.055-.038-.053-.038-.052-.038-.051-.039-.049-.039-.049-.039-.046-.039-.046-.04-.044-.04-.043-.04-.041-.04-.04-.041-.039-.041-.037-.041-.036-.041-.034-.041-.033-.042-.032-.042-.03-.042-.029-.042-.027-.042-.026-.043-.024-.043-.023-.043-.021-.043-.02-.043-.018-.044-.017-.043-.015-.044-.013-.044-.012-.044-.011-.045-.009-.044-.007-.045-.006-.045-.004-.045-.002-.045-.001-.045v-17l.001-.045.002-.045.004-.045.006-.045.007-.045.009-.044.011-.045.012-.044.013-.044.015-.044.017-.043.018-.044.02-.043.021-.043.023-.043.024-.043.026-.043.027-.042.029-.042.03-.042.032-.042.033-.042.034-.041.036-.041.037-.041.039-.041.04-.041.041-.04.043-.04.044-.04.046-.04.046-.039.049-.039.049-.039.051-.039.052-.038.053-.038.055-.038.056-.038.057-.037.058-.037.06-.037.061-.036.062-.036.063-.036.064-.036.066-.035.067-.035.068-.035.069-.035.07-.034.072-.034.072-.033.074-.033.151-.066.155-.064.159-.063.164-.061.167-.06.172-.059.176-.057.179-.056.184-.054.187-.053.19-.051.195-.05.198-.048.201-.046.204-.045.208-.043.211-.041.214-.04.217-.038.22-.036.223-.034.226-.032.228-.031.231-.028.234-.027.236-.024.238-.023.241-.02.243-.019.245-.016.247-.015.249-.012.251-.01.253-.008.255-.005.256-.004.258-.001.258.001zm-9.258 20.499v.01l.001.021.003.021.004.022.005.021.006.022.007.022.009.023.01.022.011.023.012.023.013.023.015.023.016.024.017.023.018.024.019.024.021.024.022.025.023.024.024.025.052.049.056.05.061.051.066.051.07.051.075.051.079.052.084.052.088.052.092.052.097.052.102.051.105.052.11.052.114.051.119.051.123.051.127.05.131.05.135.05.139.048.144.049.147.047.152.047.155.047.16.045.163.045.167.043.171.043.176.041.178.041.183.039.187.039.19.037.194.035.197.035.202.033.204.031.209.03.212.029.216.027.219.025.222.024.226.021.23.02.233.018.236.016.24.015.243.012.246.01.249.008.253.005.256.004.259.001.26-.001.257-.004.254-.005.25-.008.247-.011.244-.012.241-.014.237-.016.233-.018.231-.021.226-.021.224-.024.22-.026.216-.027.212-.028.21-.031.205-.031.202-.034.198-.034.194-.036.191-.037.187-.039.183-.04.179-.04.175-.042.172-.043.168-.044.163-.045.16-.046.155-.046.152-.047.148-.048.143-.049.139-.049.136-.05.131-.05.126-.05.123-.051.118-.052.114-.051.11-.052.106-.052.101-.052.096-.052.092-.052.088-.053.083-.051.079-.052.074-.052.07-.051.065-.051.06-.051.056-.05.051-.05.023-.024.023-.025.021-.024.02-.024.019-.024.018-.024.017-.024.015-.023.014-.024.013-.023.012-.023.01-.023.01-.022.008-.022.006-.022.006-.022.004-.022.004-.021.001-.021.001-.021v-4.127l-.077.055-.08.053-.083.054-.085.053-.087.052-.09.052-.093.051-.095.05-.097.05-.1.049-.102.049-.105.048-.106.047-.109.047-.111.046-.114.045-.115.045-.118.044-.12.043-.122.042-.124.042-.126.041-.128.04-.13.04-.132.038-.134.038-.135.037-.138.037-.139.035-.142.035-.143.034-.144.033-.147.032-.148.031-.15.03-.151.03-.153.029-.154.027-.156.027-.158.026-.159.025-.161.024-.162.023-.163.022-.165.021-.166.02-.167.019-.169.018-.169.017-.171.016-.173.015-.173.014-.175.013-.175.012-.177.011-.178.01-.179.008-.179.008-.181.006-.182.005-.182.004-.184.003-.184.002h-.37l-.184-.002-.184-.003-.182-.004-.182-.005-.181-.006-.179-.008-.179-.008-.178-.01-.176-.011-.176-.012-.175-.013-.173-.014-.172-.015-.171-.016-.17-.017-.169-.018-.167-.019-.166-.02-.165-.021-.163-.022-.162-.023-.161-.024-.159-.025-.157-.026-.156-.027-.155-.027-.153-.029-.151-.03-.15-.03-.148-.031-.146-.032-.145-.033-.143-.034-.141-.035-.14-.035-.137-.037-.136-.037-.134-.038-.132-.038-.13-.04-.128-.04-.126-.041-.124-.042-.122-.042-.12-.044-.117-.043-.116-.045-.113-.045-.112-.046-.109-.047-.106-.047-.105-.048-.102-.049-.1-.049-.097-.05-.095-.05-.093-.052-.09-.051-.087-.052-.085-.053-.083-.054-.08-.054-.077-.054v4.127zm0-5.654v.011l.001.021.003.021.004.021.005.022.006.022.007.022.009.022.01.022.011.023.012.023.013.023.015.024.016.023.017.024.018.024.019.024.021.024.022.024.023.025.024.024.052.05.056.05.061.05.066.051.07.051.075.052.079.051.084.052.088.052.092.052.097.052.102.052.105.052.11.051.114.051.119.052.123.05.127.051.131.05.135.049.139.049.144.048.147.048.152.047.155.046.16.045.163.045.167.044.171.042.176.042.178.04.183.04.187.038.19.037.194.036.197.034.202.033.204.032.209.03.212.028.216.027.219.025.222.024.226.022.23.02.233.018.236.016.24.014.243.012.246.01.249.008.253.006.256.003.259.001.26-.001.257-.003.254-.006.25-.008.247-.01.244-.012.241-.015.237-.016.233-.018.231-.02.226-.022.224-.024.22-.025.216-.027.212-.029.21-.03.205-.032.202-.033.198-.035.194-.036.191-.037.187-.039.183-.039.179-.041.175-.042.172-.043.168-.044.163-.045.16-.045.155-.047.152-.047.148-.048.143-.048.139-.05.136-.049.131-.05.126-.051.123-.051.118-.051.114-.052.11-.052.106-.052.101-.052.096-.052.092-.052.088-.052.083-.052.079-.052.074-.051.07-.052.065-.051.06-.05.056-.051.051-.049.023-.025.023-.024.021-.025.02-.024.019-.024.018-.024.017-.024.015-.023.014-.023.013-.024.012-.022.01-.023.01-.023.008-.022.006-.022.006-.022.004-.021.004-.022.001-.021.001-.021v-4.139l-.077.054-.08.054-.083.054-.085.052-.087.053-.09.051-.093.051-.095.051-.097.05-.1.049-.102.049-.105.048-.106.047-.109.047-.111.046-.114.045-.115.044-.118.044-.12.044-.122.042-.124.042-.126.041-.128.04-.13.039-.132.039-.134.038-.135.037-.138.036-.139.036-.142.035-.143.033-.144.033-.147.033-.148.031-.15.03-.151.03-.153.028-.154.028-.156.027-.158.026-.159.025-.161.024-.162.023-.163.022-.165.021-.166.02-.167.019-.169.018-.169.017-.171.016-.173.015-.173.014-.175.013-.175.012-.177.011-.178.009-.179.009-.179.007-.181.007-.182.005-.182.004-.184.003-.184.002h-.37l-.184-.002-.184-.003-.182-.004-.182-.005-.181-.007-.179-.007-.179-.009-.178-.009-.176-.011-.176-.012-.175-.013-.173-.014-.172-.015-.171-.016-.17-.017-.169-.018-.167-.019-.166-.02-.165-.021-.163-.022-.162-.023-.161-.024-.159-.025-.157-.026-.156-.027-.155-.028-.153-.028-.151-.03-.15-.03-.148-.031-.146-.033-.145-.033-.143-.033-.141-.035-.14-.036-.137-.036-.136-.037-.134-.038-.132-.039-.13-.039-.128-.04-.126-.041-.124-.042-.122-.043-.12-.043-.117-.044-.116-.044-.113-.046-.112-.046-.109-.046-.106-.047-.105-.048-.102-.049-.1-.049-.097-.05-.095-.051-.093-.051-.09-.051-.087-.053-.085-.052-.083-.054-.08-.054-.077-.054v4.139zm0-5.666v.011l.001.02.003.022.004.021.005.022.006.021.007.022.009.023.01.022.011.023.012.023.013.023.015.023.016.024.017.024.018.023.019.024.021.025.022.024.023.024.024.025.052.05.056.05.061.05.066.051.07.051.075.052.079.051.084.052.088.052.092.052.097.052.102.052.105.051.11.052.114.051.119.051.123.051.127.05.131.05.135.05.139.049.144.048.147.048.152.047.155.046.16.045.163.045.167.043.171.043.176.042.178.04.183.04.187.038.19.037.194.036.197.034.202.033.204.032.209.03.212.028.216.027.219.025.222.024.226.021.23.02.233.018.236.017.24.014.243.012.246.01.249.008.253.006.256.003.259.001.26-.001.257-.003.254-.006.25-.008.247-.01.244-.013.241-.014.237-.016.233-.018.231-.02.226-.022.224-.024.22-.025.216-.027.212-.029.21-.03.205-.032.202-.033.198-.035.194-.036.191-.037.187-.039.183-.039.179-.041.175-.042.172-.043.168-.044.163-.045.16-.045.155-.047.152-.047.148-.048.143-.049.139-.049.136-.049.131-.051.126-.05.123-.051.118-.052.114-.051.11-.052.106-.052.101-.052.096-.052.092-.052.088-.052.083-.052.079-.052.074-.052.07-.051.065-.051.06-.051.056-.05.051-.049.023-.025.023-.025.021-.024.02-.024.019-.024.018-.024.017-.024.015-.023.014-.024.013-.023.012-.023.01-.022.01-.023.008-.022.006-.022.006-.022.004-.022.004-.021.001-.021.001-.021v-4.153l-.077.054-.08.054-.083.053-.085.053-.087.053-.09.051-.093.051-.095.051-.097.05-.1.049-.102.048-.105.048-.106.048-.109.046-.111.046-.114.046-.115.044-.118.044-.12.043-.122.043-.124.042-.126.041-.128.04-.13.039-.132.039-.134.038-.135.037-.138.036-.139.036-.142.034-.143.034-.144.033-.147.032-.148.032-.15.03-.151.03-.153.028-.154.028-.156.027-.158.026-.159.024-.161.024-.162.023-.163.023-.165.021-.166.02-.167.019-.169.018-.169.017-.171.016-.173.015-.173.014-.175.013-.175.012-.177.01-.178.01-.179.009-.179.007-.181.006-.182.006-.182.004-.184.003-.184.001-.185.001-.185-.001-.184-.001-.184-.003-.182-.004-.182-.006-.181-.006-.179-.007-.179-.009-.178-.01-.176-.01-.176-.012-.175-.013-.173-.014-.172-.015-.171-.016-.17-.017-.169-.018-.167-.019-.166-.02-.165-.021-.163-.023-.162-.023-.161-.024-.159-.024-.157-.026-.156-.027-.155-.028-.153-.028-.151-.03-.15-.03-.148-.032-.146-.032-.145-.033-.143-.034-.141-.034-.14-.036-.137-.036-.136-.037-.134-.038-.132-.039-.13-.039-.128-.041-.126-.041-.124-.041-.122-.043-.12-.043-.117-.044-.116-.044-.113-.046-.112-.046-.109-.046-.106-.048-.105-.048-.102-.048-.1-.05-.097-.049-.095-.051-.093-.051-.09-.052-.087-.052-.085-.053-.083-.053-.08-.054-.077-.054v4.153zm8.74-8.179l-.257.004-.254.005-.25.008-.247.011-.244.012-.241.014-.237.016-.233.018-.231.021-.226.022-.224.023-.22.026-.216.027-.212.028-.21.031-.205.032-.202.033-.198.034-.194.036-.191.038-.187.038-.183.04-.179.041-.175.042-.172.043-.168.043-.163.045-.16.046-.155.046-.152.048-.148.048-.143.048-.139.049-.136.05-.131.05-.126.051-.123.051-.118.051-.114.052-.11.052-.106.052-.101.052-.096.052-.092.052-.088.052-.083.052-.079.052-.074.051-.07.052-.065.051-.06.05-.056.05-.051.05-.023.025-.023.024-.021.024-.02.025-.019.024-.018.024-.017.023-.015.024-.014.023-.013.023-.012.023-.01.023-.01.022-.008.022-.006.023-.006.021-.004.022-.004.021-.001.021-.001.021.001.021.001.021.004.021.004.022.006.021.006.023.008.022.01.022.01.023.012.023.013.023.014.023.015.024.017.023.018.024.019.024.02.025.021.024.023.024.023.025.051.05.056.05.06.05.065.051.07.052.074.051.079.052.083.052.088.052.092.052.096.052.101.052.106.052.11.052.114.052.118.051.123.051.126.051.131.05.136.05.139.049.143.048.148.048.152.048.155.046.16.046.163.045.168.043.172.043.175.042.179.041.183.04.187.038.191.038.194.036.198.034.202.033.205.032.21.031.212.028.216.027.22.026.224.023.226.022.231.021.233.018.237.016.241.014.244.012.247.011.25.008.254.005.257.004.26.001.26-.001.257-.004.254-.005.25-.008.247-.011.244-.012.241-.014.237-.016.233-.018.231-.021.226-.022.224-.023.22-.026.216-.027.212-.028.21-.031.205-.032.202-.033.198-.034.194-.036.191-.038.187-.038.183-.04.179-.041.175-.042.172-.043.168-.043.163-.045.16-.046.155-.046.152-.048.148-.048.143-.048.139-.049.136-.05.131-.05.126-.051.123-.051.118-.051.114-.052.11-.052.106-.052.101-.052.096-.052.092-.052.088-.052.083-.052.079-.052.074-.051.07-.052.065-.051.06-.05.056-.05.051-.05.023-.025.023-.024.021-.024.02-.025.019-.024.018-.024.017-.023.015-.024.014-.023.013-.023.012-.023.01-.023.01-.022.008-.022.006-.023.006-.021.004-.022.004-.021.001-.021.001-.021-.001-.021-.001-.021-.004-.021-.004-.022-.006-.021-.006-.023-.008-.022-.01-.022-.01-.023-.012-.023-.013-.023-.014-.023-.015-.024-.017-.023-.018-.024-.019-.024-.02-.025-.021-.024-.023-.024-.023-.025-.051-.05-.056-.05-.06-.05-.065-.051-.07-.052-.074-.051-.079-.052-.083-.052-.088-.052-.092-.052-.096-.052-.101-.052-.106-.052-.11-.052-.114-.052-.118-.051-.123-.051-.126-.051-.131-.05-.136-.05-.139-.049-.143-.048-.148-.048-.152-.048-.155-.046-.16-.046-.163-.045-.168-.043-.172-.043-.175-.042-.179-.041-.183-.04-.187-.038-.191-.038-.194-.036-.198-.034-.202-.033-.205-.032-.21-.031-.212-.028-.216-.027-.22-.026-.224-.023-.226-.022-.231-.021-.233-.018-.237-.016-.241-.014-.244-.012-.247-.011-.25-.008-.254-.005-.257-.004-.26-.001-.26.001z" transform="scale(.5)"/></symbol></defs><defs><symbol height="24" width="24" id="clock"><path d="M12 2c5.514 0 10 4.486 10 10s-4.486 10-10 10-10-4.486-10-10 4.486-10 10-10zm0-2c-6.627 0-12 5.373-12 12s5.373 12 12 12 12-5.373 12-12-5.373-12-12-12zm5.848 12.459c.202.038.202.333.001.372-1.907.361-6.045 1.111-6.547 1.111-.719 0-1.301-.582-1.301-1.301 0-.512.77-5.447 1.125-7.445.034-.192.312-.181.343.014l.985 6.238 5.394 1.011z" transform="scale(.5)"/></symbol></defs><g><line stroke="#999" stroke-width="0.5px" class="200" y2="622" x2="75" y1="5" x1="75" id="actor0"/><g id="root-0"><rect class="actor" ry="3" rx="3" height="65" width="150" stroke="#666" fill="#eaeaea" y="0" x="0"/><text style="text-anchor: middle; font-size: 16px; font-weight: 400;" class="actor" alignment-baseline="central" dominant-baseline="central" y="32.5" x="75"><tspan dy="0" x="75">Client</tspan></text></g></g><g><line stroke="#999" stroke-width="0.5px" class="200" y2="622" x2="736" y1="5" x1="736" id="actor1"/><g id="root-1"><rect class="actor" ry="3" rx="3" height="65" width="150" stroke="#666" fill="#eaeaea" y="0" x="661"/><text style="text-anchor: middle; font-size: 16px; font-weight: 400;" class="actor" alignment-baseline="central" dominant-baseline="central" y="32.5" x="736"><tspan dy="0" x="736">Metadata Server</tspan></text></g></g><g><line stroke="#999" stroke-width="0.5px" class="200" y2="622" x2="950.5" y1="5" x1="950.5" id="actor2"/><g id="root-2"><rect class="actor" ry="3" rx="3" height="65" width="150" stroke="#666" fill="#eaeaea" y="0" x="875.5"/><text style="text-anchor: middle; font-size: 16px; font-weight: 400;" class="actor" alignment-baseline="central" dominant-baseline="central" y="32.5" x="950.5"><tspan dy="0" x="950.5">Data Server</tspan></text></g></g><defs><marker orient="auto" markerHeight="12" markerWidth="12" markerUnits="userSpaceOnUse" refY="5" refX="9" id="arrowhead"><path d="M 0 0 L 10 5 L 0 10 z"/></marker></defs><defs><marker refY="5" refX="4" orient="auto" markerHeight="8" markerWidth="15" id="crosshead"><path style="stroke-dasharray: 0, 0;" d="M 1,2 L 6,7 M 6,2 L 1,7" stroke-width="1pt" stroke="#000000" fill="none"/></marker></defs><defs><marker orient="auto" markerHeight="28" markerWidth="20" refY="7" refX="18" id="filled-head"><path d="M 18,7 L9,13 L14,7 L9,1 Z"/></marker></defs><defs><marker orient="auto" markerHeight="40" markerWidth="60" refY="15" refX="15" id="sequencenumber"><circle r="6" cy="15" cx="15"/></marker></defs><g><rect class="note" ry="0" rx="0" height="37" width="925.5" stroke="#666" fill="#EDF2AE" y="351" x="50"/><text style="font-size: 16px; font-weight: 400;" dy="1em" class="noteText" alignment-baseline="middle" dominant-baseline="middle" text-anchor="middle" y="356" x="513"><tspan x="513">This may be parallelized</tspan></text></g><g><rect class="note" ry="0" rx="0" height="37" width="925.5" stroke="#666" fill="#EDF2AE" y="398" x="50"/><text style="font-size: 16px; font-weight: 400;" dy="1em" class="noteText" alignment-baseline="middle" dominant-baseline="middle" text-anchor="middle" y="403" x="513"><tspan x="513">Some endpoints may be processed while polling</tspan></text></g><g><line class="loopLine" y2="445" x2="960.5" y1="445" x1="65"/><line class="loopLine" y2="582" x2="960.5" y1="445" x1="960.5"/><line class="loopLine" y2="582" x2="960.5" y1="582" x1="65"/><line class="loopLine" y2="582" x2="65" y1="445" x1="65"/><polygon class="labelBox" points="65,445 115,445 115,458 106.6,465 65,465"/><text style="font-size: 16px; font-weight: 400;" class="labelText" alignment-baseline="middle" dominant-baseline="middle" text-anchor="middle" y="458" x="90">loop</text><text style="font-size: 16px; font-weight: 400;" class="loopText" text-anchor="middle" y="463" x="537.75"><tspan x="537.75">[for each endpoint in FlightInfo.endpoints]</tspan></text></g><text style="font-size: 16px; font-weight: 400;" dy="1em" class="messageText" alignment-baseline="middle" dominant-baseline="middle" text-anchor="middle" y="80" x="406">PollFlightInfo(FlightDescriptor)</text><line style="fill: none;" marker-start="url(#sequencenumber)" marker-end="url(#arrowhead)" stroke="none" stroke-width="2" class="messageLine0" y2="111" x2="736" y1="111" x1="75"/><text class="sequenceNumber" text-anchor="middle" font-size="12px" font-family="sans-serif" y="115" x="75">1</text><text style="font-size: 16px; font-weight: 400;" dy="1em" class="messageText" alignment-baseline="middle" dominant-baseline="middle" text-anchor="middle" y="126" x="406">RetryInfo{descriptor: FlightDescriptor', ...}</text><line style="fill: none;" marker-start="url(#sequencenumber)" marker-end="url(#arrowhead)" stroke="none" stroke-width="2" class="messageLine0" y2="157" x2="75" y1="157" x1="736"/><text class="sequenceNumber" text-anchor="middle" font-size="12px" font-family="sans-serif" y="161" x="736">2</text><text style="font-size: 16px; font-weight: 400;" dy="1em" class="messageText" alignment-baseline="middle" dominant-baseline="middle" text-anchor="middle" y="172" x="406">PollFlightInfo(FlightDescriptor')</text><line style="fill: none;" marker-start="url(#sequencenumber)" marker-end="url(#arrowhead)" stroke="none" stroke-width="2" class="messageLine0" y2="203" x2="736" y1="203" x1="75"/><text class="sequenceNumber" text-anchor="middle" font-size="12px" font-family="sans-serif" y="207" x="75">3</text><text style="font-size: 16px; font-weight: 400;" dy="1em" class="messageText" alignment-baseline="middle" dominant-baseline="middle" text-anchor="middle" y="218" x="406">RetryInfo{descriptor: FlightDescriptor'', ...}</text><line style="fill: none;" marker-start="url(#sequencenumber)" marker-end="url(#arrowhead)" stroke="none" stroke-width="2" class="messageLine0" y2="249" x2="75" y1="249" x1="736"/><text class="sequenceNumber" text-anchor="middle" font-size="12px" font-family="sans-serif" y="253" x="736">4</text><text style="font-size: 16px; font-weight: 400;" dy="1em" class="messageText" alignment-baseline="middle" dominant-baseline="middle" text-anchor="middle" y="264" x="406">PollFlightInfo(FlightDescriptor'')</text><line style="fill: none;" marker-start="url(#sequencenumber)" marker-end="url(#arrowhead)" stroke="none" stroke-width="2" class="messageLine0" y2="295" x2="736" y1="295" x1="75"/><text class="sequenceNumber" text-anchor="middle" font-size="12px" font-family="sans-serif" y="299" x="75">5</text><text style="font-size: 16px; font-weight: 400;" dy="1em" class="messageText" alignment-baseline="middle" dominant-baseline="middle" text-anchor="middle" y="310" x="406">RetryInfo{descriptor: null, info: FlightInfo{endpoints: [FlightEndpoint{ticket: Ticket}, …]}</text><line style="fill: none;" marker-start="url(#sequencenumber)" marker-end="url(#arrowhead)" stroke="none" stroke-width="2" class="messageLine0" y2="341" x2="75" y1="341" x1="736"/><text class="sequenceNumber" text-anchor="middle" font-size="12px" font-family="sans-serif" y="345" x="736">6</text><text style="font-size: 16px; font-weight: 400;" dy="1em" class="messageText" alignment-baseline="middle" dominant-baseline="middle" text-anchor="middle" y="495" x="513">DoGet(Ticket)</text><line style="fill: none;" marker-start="url(#sequencenumber)" marker-end="url(#arrowhead)" stroke="none" stroke-width="2" class="messageLine0" y2="526" x2="950.5" y1="526" x1="75"/><text class="sequenceNumber" text-anchor="middle" font-size="12px" font-family="sans-serif" y="530" x="75">7</text><text style="font-size: 16px; font-weight: 400;" dy="1em" class="messageText" alignment-baseline="middle" dominant-baseline="middle" text-anchor="middle" y="541" x="513">stream of FlightData</text><line style="fill: none;" marker-start="url(#sequencenumber)" marker-end="url(#arrowhead)" stroke="none" stroke-width="2" class="messageLine0" y2="572" x2="75" y1="572" x1="950.5"/><text class="sequenceNumber" text-anchor="middle" font-size="12px" font-family="sans-serif" y="576" x="950.5">8</text><g><rect class="actor" ry="3" rx="3" height="65" width="150" stroke="#666" fill="#eaeaea" y="602" x="0"/><text style="text-anchor: middle; font-size: 16px; font-weight: 400;" class="actor" alignment-baseline="central" dominant-baseline="central" y="634.5" x="75"><tspan dy="0" x="75">Client</tspan></text></g><g><rect class="actor" ry="3" rx="3" height="65" width="150" stroke="#666" fill="#eaeaea" y="602" x="661"/><text style="text-anchor: middle; font-size: 16px; font-weight: 400;" class="actor" alignment-baseline="central" dominant-baseline="central" y="634.5" x="736"><tspan dy="0" x="736">Metadata Server</tspan></text></g><g><rect class="actor" ry="3" rx="3" height="65" width="150" stroke="#666" fill="#eaeaea" y="602" x="875.5"/><text style="text-anchor: middle; font-size: 16px; font-weight: 400;" class="actor" alignment-baseline="central" dominant-baseline="central" y="634.5" x="950.5"><tspan dy="0" x="950.5">Data Server</tspan></text></g></svg> \ No newline at end of file diff --git a/format/Flight.proto b/format/Flight.proto index b5d23f6f7e6d2..baf2733d81048 100644 --- a/format/Flight.proto +++ b/format/Flight.proto @@ -65,6 +65,32 @@ service FlightService { */ rpc GetFlightInfo(FlightDescriptor) returns (FlightInfo) {} + /* + * 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 + * should return as quickly as possible. (GetFlightInfo doesn't + * return until the query is complete.) + * + * A client can consume any available results before + * the query is completed. See PollInfo.info for details. + * + * A client can poll the updated query status by calling + * PollFlightInfo() with PollInfo.flight_descriptor. A server + * should not respond until the result would be different from last + * time. That way, the client can "long poll" for updates + * without constantly making requests. Clients can set a short timeout + * to avoid blocking calls if desired. + * + * A client can't use PollInfo.flight_descriptor after + * PollInfo.expiration_time passes. A server might not accept the + * retry descriptor anymore and the query may be cancelled. + * + * A client may use the CancelFlightInfo action with + * PollInfo.info to cancel the running query. + */ + rpc PollFlightInfo(FlightDescriptor) returns (PollInfo) {} + /* * 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 @@ -116,7 +142,6 @@ service FlightService { * understand the capabilities of the flight service. */ rpc ListActions(Empty) returns (stream ActionType) {} - } /* @@ -343,6 +368,50 @@ message FlightInfo { bool ordered = 6; } +/* + * The information to process a long-running query. + */ +message PollInfo { + /* + * The currently available results. + * + * If "flight_descriptor" is not specified, the query is complete + * and "info" specifies all results. Otherwise, "info" contains + * partial query results. + * + * Note that each PollInfo response contains a complete + * FlightInfo (not just the delta between the previous and current + * FlightInfo). + * + * Subsequent PollInfo responses may only append new endpoints to + * info. + * + * Clients can begin fetching results via DoGet(Ticket) with the + * ticket in the info before the query is + * completed. FlightInfo.ordered is also valid. + */ + FlightInfo info = 1; + + /* + * The descriptor the client should use on the next try. + * If unset, the query is complete. + */ + FlightDescriptor flight_descriptor = 2; + + /* + * Query progress. If known, must be in [0.0, 1.0] but need not be + * monotonic or nondecreasing. If unknown, do not set. + */ + optional double progress = 3; + + /* + * 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. + */ + google.protobuf.Timestamp expiration_time = 4; +} + /* * A particular stream or split associated with a flight. */ diff --git a/go/arrow/flight/flightsql/client_test.go b/go/arrow/flight/flightsql/client_test.go index 1532ef5f32f54..fdc08a247f9ff 100644 --- a/go/arrow/flight/flightsql/client_test.go +++ b/go/arrow/flight/flightsql/client_test.go @@ -87,6 +87,11 @@ func (m *FlightServiceClientMock) GetFlightInfo(ctx context.Context, in *flight. return args.Get(0).(*flight.FlightInfo), args.Error(1) } +func (m *FlightServiceClientMock) PollFlightInfo(ctx context.Context, in *flight.FlightDescriptor, opts ...grpc.CallOption) (*flight.PollInfo, error) { + args := m.Called(in.Type, in.Cmd, opts) + return args.Get(0).(*flight.PollInfo), args.Error(1) +} + func (m *FlightServiceClientMock) GetSchema(ctx context.Context, in *flight.FlightDescriptor, opts ...grpc.CallOption) (*flight.SchemaResult, error) { panic("not implemented") // TODO: Implement } diff --git a/go/arrow/flight/gen/flight/Flight.pb.go b/go/arrow/flight/gen/flight/Flight.pb.go index f91d762014603..861b37d90d088 100644 --- a/go/arrow/flight/gen/flight/Flight.pb.go +++ b/go/arrow/flight/gen/flight/Flight.pb.go @@ -18,15 +18,15 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.28.1 -// protoc v3.12.4 +// protoc v3.21.12 // source: Flight.proto package flight import ( - timestamp "github.com/golang/protobuf/ptypes/timestamp" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" reflect "reflect" sync "sync" ) @@ -38,7 +38,6 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -// // The result of a cancel operation. // // This is used by CancelFlightInfoResult.status. @@ -103,19 +102,17 @@ 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. - FlightDescriptor_PATH FlightDescriptor_DescriptorType = 1 // + // similar to a path inside a filesystem. + FlightDescriptor_PATH FlightDescriptor_DescriptorType = 1 // An opaque command to generate a dataset. FlightDescriptor_CMD FlightDescriptor_DescriptorType = 2 ) @@ -161,17 +158,14 @@ 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"` } @@ -227,10 +221,8 @@ 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"` } @@ -281,7 +273,6 @@ func (x *HandshakeResponse) GetPayload() []byte { return nil } -// // A message for doing simple auth. type BasicAuth struct { state protoimpl.MessageState @@ -376,7 +367,6 @@ 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 { @@ -434,7 +424,6 @@ 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 { @@ -484,7 +473,6 @@ func (x *Criteria) GetExpression() []byte { return nil } -// // An opaque action specific for the service. type Action struct { state protoimpl.MessageState @@ -541,7 +529,6 @@ func (x *Action) GetBody() []byte { return nil } -// // The request of the CancelFlightInfo action. // // The request should be stored in Action.body. @@ -592,7 +579,6 @@ func (x *CancelFlightInfoRequest) GetInfo() *FlightInfo { return nil } -// // The request of the RenewFlightEndpoint action. // // The request should be stored in Action.body. @@ -643,7 +629,6 @@ func (x *RenewFlightEndpointRequest) GetEndpoint() *FlightEndpoint { return nil } -// // An opaque result returned after executing an action. type Result struct { state protoimpl.MessageState @@ -692,7 +677,6 @@ func (x *Result) GetBody() []byte { return nil } -// // The result of the CancelFlightInfo action. // // The result should be stored in Result.body. @@ -743,7 +727,6 @@ func (x *CancelFlightInfoResult) GetStatus() CancelStatus { return CancelStatus_CANCEL_STATUS_UNSPECIFIED } -// // Wrap the result of a getSchema call type SchemaResult struct { state protoimpl.MessageState @@ -751,9 +734,10 @@ 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"` } @@ -796,7 +780,6 @@ 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 { @@ -805,11 +788,9 @@ 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"` @@ -868,7 +849,6 @@ 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 { @@ -877,14 +857,13 @@ 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 - Schema []byte `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"` // + // 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. @@ -904,14 +883,13 @@ 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"` } @@ -990,17 +968,109 @@ func (x *FlightInfo) GetOrdered() bool { return false } -// +// 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 + // and "info" specifies all results. Otherwise, "info" contains + // partial query results. + // + // Note that each PollInfo response contains a complete + // FlightInfo (not just the delta between the previous and current + // FlightInfo). + // + // Subsequent PollInfo responses may only append new endpoints to + // info. + // + // Clients can begin fetching results via DoGet(Ticket) with the + // 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. + ExpirationTime *timestamppb.Timestamp `protobuf:"bytes,4,opt,name=expiration_time,json=expirationTime,proto3" json:"expiration_time,omitempty"` +} + +func (x *PollInfo) Reset() { + *x = PollInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_Flight_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PollInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PollInfo) ProtoMessage() {} + +func (x *PollInfo) ProtoReflect() protoreflect.Message { + mi := &file_Flight_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PollInfo.ProtoReflect.Descriptor instead. +func (*PollInfo) Descriptor() ([]byte, []int) { + return file_Flight_proto_rawDescGZIP(), []int{14} +} + +func (x *PollInfo) GetInfo() *FlightInfo { + if x != nil { + return x.Info + } + return nil +} + +func (x *PollInfo) GetFlightDescriptor() *FlightDescriptor { + if x != nil { + return x.FlightDescriptor + } + return nil +} + +func (x *PollInfo) GetProgress() float64 { + if x != nil && x.Progress != nil { + return *x.Progress + } + return 0 +} + +func (x *PollInfo) GetExpirationTime() *timestamppb.Timestamp { + if x != nil { + return x.ExpirationTime + } + 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 @@ -1016,17 +1086,16 @@ 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 *timestamp.Timestamp `protobuf:"bytes,3,opt,name=expiration_time,json=expirationTime,proto3" json:"expiration_time,omitempty"` + ExpirationTime *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=expiration_time,json=expirationTime,proto3" json:"expiration_time,omitempty"` } func (x *FlightEndpoint) Reset() { *x = FlightEndpoint{} if protoimpl.UnsafeEnabled { - mi := &file_Flight_proto_msgTypes[14] + mi := &file_Flight_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1039,7 +1108,7 @@ func (x *FlightEndpoint) String() string { func (*FlightEndpoint) ProtoMessage() {} func (x *FlightEndpoint) ProtoReflect() protoreflect.Message { - mi := &file_Flight_proto_msgTypes[14] + mi := &file_Flight_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1052,7 +1121,7 @@ func (x *FlightEndpoint) ProtoReflect() protoreflect.Message { // Deprecated: Use FlightEndpoint.ProtoReflect.Descriptor instead. func (*FlightEndpoint) Descriptor() ([]byte, []int) { - return file_Flight_proto_rawDescGZIP(), []int{14} + return file_Flight_proto_rawDescGZIP(), []int{15} } func (x *FlightEndpoint) GetTicket() *Ticket { @@ -1069,14 +1138,13 @@ func (x *FlightEndpoint) GetLocation() []*Location { return nil } -func (x *FlightEndpoint) GetExpirationTime() *timestamp.Timestamp { +func (x *FlightEndpoint) GetExpirationTime() *timestamppb.Timestamp { if x != nil { return x.ExpirationTime } return nil } -// // A location where a Flight service will accept retrieval of a particular // stream given a ticket. type Location struct { @@ -1090,7 +1158,7 @@ type Location struct { func (x *Location) Reset() { *x = Location{} if protoimpl.UnsafeEnabled { - mi := &file_Flight_proto_msgTypes[15] + mi := &file_Flight_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1103,7 +1171,7 @@ func (x *Location) String() string { func (*Location) ProtoMessage() {} func (x *Location) ProtoReflect() protoreflect.Message { - mi := &file_Flight_proto_msgTypes[15] + mi := &file_Flight_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1116,7 +1184,7 @@ func (x *Location) ProtoReflect() protoreflect.Message { // Deprecated: Use Location.ProtoReflect.Descriptor instead. func (*Location) Descriptor() ([]byte, []int) { - return file_Flight_proto_rawDescGZIP(), []int{15} + return file_Flight_proto_rawDescGZIP(), []int{16} } func (x *Location) GetUri() string { @@ -1126,7 +1194,6 @@ func (x *Location) GetUri() string { return "" } -// // An opaque identifier that the service can use to retrieve a particular // portion of a stream. // @@ -1143,7 +1210,7 @@ type Ticket struct { func (x *Ticket) Reset() { *x = Ticket{} if protoimpl.UnsafeEnabled { - mi := &file_Flight_proto_msgTypes[16] + mi := &file_Flight_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1156,7 +1223,7 @@ func (x *Ticket) String() string { func (*Ticket) ProtoMessage() {} func (x *Ticket) ProtoReflect() protoreflect.Message { - mi := &file_Flight_proto_msgTypes[16] + mi := &file_Flight_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1169,7 +1236,7 @@ func (x *Ticket) ProtoReflect() protoreflect.Message { // Deprecated: Use Ticket.ProtoReflect.Descriptor instead. func (*Ticket) Descriptor() ([]byte, []int) { - return file_Flight_proto_rawDescGZIP(), []int{16} + return file_Flight_proto_rawDescGZIP(), []int{17} } func (x *Ticket) GetTicket() []byte { @@ -1179,24 +1246,19 @@ 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 @@ -1207,7 +1269,7 @@ type FlightData struct { func (x *FlightData) Reset() { *x = FlightData{} if protoimpl.UnsafeEnabled { - mi := &file_Flight_proto_msgTypes[17] + mi := &file_Flight_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1220,7 +1282,7 @@ func (x *FlightData) String() string { func (*FlightData) ProtoMessage() {} func (x *FlightData) ProtoReflect() protoreflect.Message { - mi := &file_Flight_proto_msgTypes[17] + mi := &file_Flight_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1233,7 +1295,7 @@ func (x *FlightData) ProtoReflect() protoreflect.Message { // Deprecated: Use FlightData.ProtoReflect.Descriptor instead. func (*FlightData) Descriptor() ([]byte, []int) { - return file_Flight_proto_rawDescGZIP(), []int{17} + return file_Flight_proto_rawDescGZIP(), []int{18} } func (x *FlightData) GetFlightDescriptor() *FlightDescriptor { @@ -1264,7 +1326,7 @@ func (x *FlightData) GetDataBody() []byte { return nil } -//* +// * // The response message associated with the submission of a DoPut. type PutResult struct { state protoimpl.MessageState @@ -1277,7 +1339,7 @@ type PutResult struct { func (x *PutResult) Reset() { *x = PutResult{} if protoimpl.UnsafeEnabled { - mi := &file_Flight_proto_msgTypes[18] + mi := &file_Flight_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1290,7 +1352,7 @@ func (x *PutResult) String() string { func (*PutResult) ProtoMessage() {} func (x *PutResult) ProtoReflect() protoreflect.Message { - mi := &file_Flight_proto_msgTypes[18] + mi := &file_Flight_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1303,7 +1365,7 @@ func (x *PutResult) ProtoReflect() protoreflect.Message { // Deprecated: Use PutResult.ProtoReflect.Descriptor instead. func (*PutResult) Descriptor() ([]byte, []int) { - return file_Flight_proto_rawDescGZIP(), []int{18} + return file_Flight_proto_rawDescGZIP(), []int{19} } func (x *PutResult) GetAppMetadata() []byte { @@ -1397,106 +1459,129 @@ 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, 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, 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, 0xa7, 0x06, 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, + 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, 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, 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, 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, + 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, + 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, - 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, + 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, 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, 0x76, 0x0a, - 0x1c, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x61, 0x72, 0x72, 0x6f, - 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x69, 0x6d, 0x70, 0x6c, 0x5a, 0x37, 0x67, - 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, - 0x2f, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2f, 0x67, 0x6f, 0x2f, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2f, - 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, - 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 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, + 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, 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, } var ( @@ -1512,7 +1597,7 @@ func file_Flight_proto_rawDescGZIP() []byte { } var file_Flight_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_Flight_proto_msgTypes = make([]protoimpl.MessageInfo, 19) +var file_Flight_proto_msgTypes = make([]protoimpl.MessageInfo, 20) var file_Flight_proto_goTypes = []interface{}{ (CancelStatus)(0), // 0: arrow.flight.protocol.CancelStatus (FlightDescriptor_DescriptorType)(0), // 1: arrow.flight.protocol.FlightDescriptor.DescriptorType @@ -1530,47 +1615,53 @@ var file_Flight_proto_goTypes = []interface{}{ (*SchemaResult)(nil), // 13: arrow.flight.protocol.SchemaResult (*FlightDescriptor)(nil), // 14: arrow.flight.protocol.FlightDescriptor (*FlightInfo)(nil), // 15: arrow.flight.protocol.FlightInfo - (*FlightEndpoint)(nil), // 16: arrow.flight.protocol.FlightEndpoint - (*Location)(nil), // 17: arrow.flight.protocol.Location - (*Ticket)(nil), // 18: arrow.flight.protocol.Ticket - (*FlightData)(nil), // 19: arrow.flight.protocol.FlightData - (*PutResult)(nil), // 20: arrow.flight.protocol.PutResult - (*timestamp.Timestamp)(nil), // 21: google.protobuf.Timestamp + (*PollInfo)(nil), // 16: arrow.flight.protocol.PollInfo + (*FlightEndpoint)(nil), // 17: arrow.flight.protocol.FlightEndpoint + (*Location)(nil), // 18: arrow.flight.protocol.Location + (*Ticket)(nil), // 19: arrow.flight.protocol.Ticket + (*FlightData)(nil), // 20: arrow.flight.protocol.FlightData + (*PutResult)(nil), // 21: arrow.flight.protocol.PutResult + (*timestamppb.Timestamp)(nil), // 22: google.protobuf.Timestamp } var file_Flight_proto_depIdxs = []int32{ 15, // 0: arrow.flight.protocol.CancelFlightInfoRequest.info:type_name -> arrow.flight.protocol.FlightInfo - 16, // 1: arrow.flight.protocol.RenewFlightEndpointRequest.endpoint:type_name -> arrow.flight.protocol.FlightEndpoint + 17, // 1: arrow.flight.protocol.RenewFlightEndpointRequest.endpoint:type_name -> arrow.flight.protocol.FlightEndpoint 0, // 2: arrow.flight.protocol.CancelFlightInfoResult.status:type_name -> arrow.flight.protocol.CancelStatus 1, // 3: arrow.flight.protocol.FlightDescriptor.type:type_name -> arrow.flight.protocol.FlightDescriptor.DescriptorType 14, // 4: arrow.flight.protocol.FlightInfo.flight_descriptor:type_name -> arrow.flight.protocol.FlightDescriptor - 16, // 5: arrow.flight.protocol.FlightInfo.endpoint:type_name -> arrow.flight.protocol.FlightEndpoint - 18, // 6: arrow.flight.protocol.FlightEndpoint.ticket:type_name -> arrow.flight.protocol.Ticket - 17, // 7: arrow.flight.protocol.FlightEndpoint.location:type_name -> arrow.flight.protocol.Location - 21, // 8: arrow.flight.protocol.FlightEndpoint.expiration_time:type_name -> google.protobuf.Timestamp - 14, // 9: arrow.flight.protocol.FlightData.flight_descriptor:type_name -> arrow.flight.protocol.FlightDescriptor - 2, // 10: arrow.flight.protocol.FlightService.Handshake:input_type -> arrow.flight.protocol.HandshakeRequest - 7, // 11: arrow.flight.protocol.FlightService.ListFlights:input_type -> arrow.flight.protocol.Criteria - 14, // 12: arrow.flight.protocol.FlightService.GetFlightInfo:input_type -> arrow.flight.protocol.FlightDescriptor - 14, // 13: arrow.flight.protocol.FlightService.GetSchema:input_type -> arrow.flight.protocol.FlightDescriptor - 18, // 14: arrow.flight.protocol.FlightService.DoGet:input_type -> arrow.flight.protocol.Ticket - 19, // 15: arrow.flight.protocol.FlightService.DoPut:input_type -> arrow.flight.protocol.FlightData - 19, // 16: arrow.flight.protocol.FlightService.DoExchange:input_type -> arrow.flight.protocol.FlightData - 8, // 17: arrow.flight.protocol.FlightService.DoAction:input_type -> arrow.flight.protocol.Action - 5, // 18: arrow.flight.protocol.FlightService.ListActions:input_type -> arrow.flight.protocol.Empty - 3, // 19: arrow.flight.protocol.FlightService.Handshake:output_type -> arrow.flight.protocol.HandshakeResponse - 15, // 20: arrow.flight.protocol.FlightService.ListFlights:output_type -> arrow.flight.protocol.FlightInfo - 15, // 21: arrow.flight.protocol.FlightService.GetFlightInfo:output_type -> arrow.flight.protocol.FlightInfo - 13, // 22: arrow.flight.protocol.FlightService.GetSchema:output_type -> arrow.flight.protocol.SchemaResult - 19, // 23: arrow.flight.protocol.FlightService.DoGet:output_type -> arrow.flight.protocol.FlightData - 20, // 24: arrow.flight.protocol.FlightService.DoPut:output_type -> arrow.flight.protocol.PutResult - 19, // 25: arrow.flight.protocol.FlightService.DoExchange:output_type -> arrow.flight.protocol.FlightData - 11, // 26: arrow.flight.protocol.FlightService.DoAction:output_type -> arrow.flight.protocol.Result - 6, // 27: arrow.flight.protocol.FlightService.ListActions:output_type -> arrow.flight.protocol.ActionType - 19, // [19:28] is the sub-list for method output_type - 10, // [10:19] is the sub-list for method input_type - 10, // [10:10] is the sub-list for extension type_name - 10, // [10:10] is the sub-list for extension extendee - 0, // [0:10] is the sub-list for field type_name + 17, // 5: arrow.flight.protocol.FlightInfo.endpoint:type_name -> arrow.flight.protocol.FlightEndpoint + 15, // 6: arrow.flight.protocol.PollInfo.info:type_name -> arrow.flight.protocol.FlightInfo + 14, // 7: arrow.flight.protocol.PollInfo.flight_descriptor:type_name -> arrow.flight.protocol.FlightDescriptor + 22, // 8: arrow.flight.protocol.PollInfo.expiration_time:type_name -> google.protobuf.Timestamp + 19, // 9: arrow.flight.protocol.FlightEndpoint.ticket:type_name -> arrow.flight.protocol.Ticket + 18, // 10: arrow.flight.protocol.FlightEndpoint.location:type_name -> arrow.flight.protocol.Location + 22, // 11: arrow.flight.protocol.FlightEndpoint.expiration_time:type_name -> google.protobuf.Timestamp + 14, // 12: arrow.flight.protocol.FlightData.flight_descriptor:type_name -> arrow.flight.protocol.FlightDescriptor + 2, // 13: arrow.flight.protocol.FlightService.Handshake:input_type -> arrow.flight.protocol.HandshakeRequest + 7, // 14: arrow.flight.protocol.FlightService.ListFlights:input_type -> arrow.flight.protocol.Criteria + 14, // 15: arrow.flight.protocol.FlightService.GetFlightInfo:input_type -> arrow.flight.protocol.FlightDescriptor + 14, // 16: arrow.flight.protocol.FlightService.PollFlightInfo:input_type -> arrow.flight.protocol.FlightDescriptor + 14, // 17: arrow.flight.protocol.FlightService.GetSchema:input_type -> arrow.flight.protocol.FlightDescriptor + 19, // 18: arrow.flight.protocol.FlightService.DoGet:input_type -> arrow.flight.protocol.Ticket + 20, // 19: arrow.flight.protocol.FlightService.DoPut:input_type -> arrow.flight.protocol.FlightData + 20, // 20: arrow.flight.protocol.FlightService.DoExchange:input_type -> arrow.flight.protocol.FlightData + 8, // 21: arrow.flight.protocol.FlightService.DoAction:input_type -> arrow.flight.protocol.Action + 5, // 22: arrow.flight.protocol.FlightService.ListActions:input_type -> arrow.flight.protocol.Empty + 3, // 23: arrow.flight.protocol.FlightService.Handshake:output_type -> arrow.flight.protocol.HandshakeResponse + 15, // 24: arrow.flight.protocol.FlightService.ListFlights:output_type -> arrow.flight.protocol.FlightInfo + 15, // 25: arrow.flight.protocol.FlightService.GetFlightInfo:output_type -> arrow.flight.protocol.FlightInfo + 16, // 26: arrow.flight.protocol.FlightService.PollFlightInfo:output_type -> arrow.flight.protocol.PollInfo + 13, // 27: arrow.flight.protocol.FlightService.GetSchema:output_type -> arrow.flight.protocol.SchemaResult + 20, // 28: arrow.flight.protocol.FlightService.DoGet:output_type -> arrow.flight.protocol.FlightData + 21, // 29: arrow.flight.protocol.FlightService.DoPut:output_type -> arrow.flight.protocol.PutResult + 20, // 30: arrow.flight.protocol.FlightService.DoExchange:output_type -> arrow.flight.protocol.FlightData + 11, // 31: arrow.flight.protocol.FlightService.DoAction:output_type -> arrow.flight.protocol.Result + 6, // 32: arrow.flight.protocol.FlightService.ListActions:output_type -> arrow.flight.protocol.ActionType + 23, // [23:33] is the sub-list for method output_type + 13, // [13:23] is the sub-list for method input_type + 13, // [13:13] is the sub-list for extension type_name + 13, // [13:13] is the sub-list for extension extendee + 0, // [0:13] is the sub-list for field type_name } func init() { file_Flight_proto_init() } @@ -1748,7 +1839,7 @@ func file_Flight_proto_init() { } } file_Flight_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FlightEndpoint); i { + switch v := v.(*PollInfo); i { case 0: return &v.state case 1: @@ -1760,7 +1851,7 @@ func file_Flight_proto_init() { } } file_Flight_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Location); i { + switch v := v.(*FlightEndpoint); i { case 0: return &v.state case 1: @@ -1772,7 +1863,7 @@ func file_Flight_proto_init() { } } file_Flight_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Ticket); i { + switch v := v.(*Location); i { case 0: return &v.state case 1: @@ -1784,7 +1875,7 @@ func file_Flight_proto_init() { } } file_Flight_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FlightData); i { + switch v := v.(*Ticket); i { case 0: return &v.state case 1: @@ -1796,6 +1887,18 @@ func file_Flight_proto_init() { } } file_Flight_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FlightData); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_Flight_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PutResult); i { case 0: return &v.state @@ -1808,13 +1911,14 @@ func file_Flight_proto_init() { } } } + file_Flight_proto_msgTypes[14].OneofWrappers = []interface{}{} type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_Flight_proto_rawDesc, NumEnums: 2, - NumMessages: 19, + NumMessages: 20, NumExtensions: 0, NumServices: 1, }, diff --git a/go/arrow/flight/gen/flight/FlightSql.pb.go b/go/arrow/flight/gen/flight/FlightSql.pb.go index 3b1ba232d3d12..95faa719a9488 100644 --- a/go/arrow/flight/gen/flight/FlightSql.pb.go +++ b/go/arrow/flight/gen/flight/FlightSql.pb.go @@ -18,15 +18,15 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.28.1 -// protoc v3.12.4 +// protoc v3.21.12 // source: FlightSql.proto package flight import ( - descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" + descriptorpb "google.golang.org/protobuf/types/descriptorpb" reflect "reflect" sync "sync" ) @@ -48,33 +48,27 @@ 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. // @@ -84,61 +78,51 @@ 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 @@ -154,18 +138,15 @@ 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. @@ -174,7 +155,6 @@ 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. // @@ -182,13 +162,11 @@ 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<int32, list<int32>> 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. // @@ -196,7 +174,6 @@ 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. // @@ -204,14 +181,12 @@ 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. // @@ -219,7 +194,6 @@ 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. @@ -232,21 +206,18 @@ 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. @@ -263,7 +234,6 @@ 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. @@ -280,14 +250,12 @@ 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 @@ -297,17 +265,14 @@ 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. @@ -324,7 +289,6 @@ 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. @@ -341,7 +305,6 @@ 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. @@ -354,14 +317,12 @@ 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. // @@ -369,7 +330,6 @@ 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. @@ -395,14 +355,12 @@ 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. @@ -435,7 +393,6 @@ 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 @@ -447,17 +404,15 @@ 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 @@ -469,13 +424,11 @@ 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`. // @@ -483,7 +436,6 @@ 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. @@ -510,7 +462,6 @@ 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. // @@ -518,14 +469,12 @@ 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. // @@ -542,7 +491,6 @@ 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`. // @@ -557,7 +505,6 @@ 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`. // @@ -572,7 +519,6 @@ 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`. // @@ -587,7 +533,6 @@ 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`. // @@ -602,34 +547,29 @@ 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. // @@ -1702,7 +1642,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 @@ -1817,7 +1757,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 @@ -1958,13 +1898,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 ) @@ -2013,21 +1953,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 ) @@ -2293,22 +2233,23 @@ 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<string_data: utf8> -// int32_to_int32_list_map: map<key: int32, value: list<$data$: int32>> +// +// info_name: uint32 not null, +// value: dense_union< +// string_value: utf8, +// bool_value: bool, +// bigint_value: int64, +// int32_bitmask: int32, +// string_list: list<string_data: utf8> +// int32_to_int32_list_map: map<key: int32, value: list<$data$: int32>> +// // > // where there is one row per requested piece of metadata information. type CommandGetSqlInfo struct { @@ -2316,7 +2257,6 @@ 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. @@ -2376,61 +2316,62 @@ 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<utf8 not null> -// (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<utf8 not null> +// (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 { @@ -2438,7 +2379,6 @@ 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"` } @@ -2482,16 +2422,17 @@ 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 { @@ -2532,17 +2473,18 @@ 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 { @@ -2550,17 +2492,15 @@ 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"` } @@ -2610,58 +2550,56 @@ 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. @@ -2737,17 +2675,18 @@ 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 { @@ -2788,20 +2727,21 @@ 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 { @@ -2809,12 +2749,10 @@ 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. @@ -2876,28 +2814,29 @@ 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. @@ -2906,12 +2845,10 @@ 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. @@ -2973,46 +2910,45 @@ 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. @@ -3074,66 +3010,67 @@ 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"` } @@ -3212,7 +3149,6 @@ func (x *CommandGetCrossReference) GetFkTable() string { return "" } -// // Request message for the "CreatePreparedStatement" action on a Flight SQL enabled backend. type ActionCreatePreparedStatementRequest struct { state protoimpl.MessageState @@ -3272,7 +3208,6 @@ func (x *ActionCreatePreparedStatementRequest) GetTransactionId() []byte { return nil } -// // An embedded message describing a Substrait plan to execute. type SubstraitPlan struct { state protoimpl.MessageState @@ -3336,7 +3271,6 @@ func (x *SubstraitPlan) GetVersion() string { return "" } -// // Request message for the "CreatePreparedSubstraitPlan" action on a Flight SQL enabled backend. type ActionCreatePreparedSubstraitPlanRequest struct { state protoimpl.MessageState @@ -3396,7 +3330,6 @@ func (x *ActionCreatePreparedSubstraitPlanRequest) GetTransactionId() []byte { return nil } -// // Wrap the result of a "CreatePreparedStatement" or "CreatePreparedSubstraitPlan" action. // // The resultant PreparedStatement can be closed either: @@ -3472,7 +3405,6 @@ 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 { @@ -3523,7 +3455,6 @@ func (x *ActionClosePreparedStatementRequest) GetPreparedStatementHandle() []byt return nil } -// // Request message for the "BeginTransaction" action. // Begins a transaction. type ActionBeginTransactionRequest struct { @@ -3564,7 +3495,6 @@ func (*ActionBeginTransactionRequest) Descriptor() ([]byte, []int) { return file_FlightSql_proto_rawDescGZIP(), []int{15} } -// // Request message for the "BeginSavepoint" action. // Creates a savepoint within a transaction. // @@ -3627,7 +3557,6 @@ func (x *ActionBeginSavepointRequest) GetName() string { return "" } -// // The result of a "BeginTransaction" action. // // The transaction can be manipulated with the "EndTransaction" action, or @@ -3683,7 +3612,6 @@ func (x *ActionBeginTransactionResult) GetTransactionId() []byte { return nil } -// // The result of a "BeginSavepoint" action. // // The transaction can be manipulated with the "EndSavepoint" action. @@ -3739,7 +3667,6 @@ func (x *ActionBeginSavepointResult) GetSavepointId() []byte { return nil } -// // Request message for the "EndTransaction" action. // // Commit (COMMIT) or rollback (ROLLBACK) the transaction. @@ -3803,7 +3730,6 @@ 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 @@ -3869,22 +3795,21 @@ 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 @@ -3942,23 +3867,22 @@ 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 @@ -4016,7 +3940,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 { @@ -4067,23 +3991,22 @@ 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 @@ -4132,7 +4055,6 @@ 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 { @@ -4192,7 +4114,6 @@ 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. @@ -4244,7 +4165,6 @@ 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. @@ -4297,7 +4217,6 @@ func (x *DoPutUpdateResult) GetRecordCount() int64 { return 0 } -// // Request message for the "CancelQuery" action. // // Explicitly cancel a running query. @@ -4366,7 +4285,6 @@ func (x *ActionCancelQueryRequest) GetInfo() []byte { return nil } -// // The result of cancelling a query. // // The result should be wrapped in a google.protobuf.Any message. @@ -4424,7 +4342,7 @@ func (x *ActionCancelQueryResult) GetResult() ActionCancelQueryResult_CancelResu var file_FlightSql_proto_extTypes = []protoimpl.ExtensionInfo{ { - ExtendedType: (*descriptor.MessageOptions)(nil), + ExtendedType: (*descriptorpb.MessageOptions)(nil), ExtensionType: (*bool)(nil), Field: 1000, Name: "arrow.flight.protocol.sql.experimental", @@ -4433,7 +4351,7 @@ var file_FlightSql_proto_extTypes = []protoimpl.ExtensionInfo{ }, } -// Extension fields to descriptor.MessageOptions. +// Extension fields to descriptorpb.MessageOptions. var ( // optional bool experimental = 1000; E_Experimental = &file_FlightSql_proto_extTypes[0] @@ -5169,13 +5087,13 @@ var file_FlightSql_proto_rawDesc = []byte{ 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xe8, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x65, 0x78, 0x70, 0x65, 0x72, 0x69, 0x6d, 0x65, 0x6e, 0x74, - 0x61, 0x6c, 0x42, 0x5b, 0x0a, 0x20, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x61, 0x6c, 0x42, 0x56, 0x0a, 0x20, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2e, 0x73, 0x71, - 0x6c, 0x2e, 0x69, 0x6d, 0x70, 0x6c, 0x5a, 0x37, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6c, 0x2e, 0x69, 0x6d, 0x70, 0x6c, 0x5a, 0x32, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2f, 0x67, 0x6f, 0x2f, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x2f, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x2f, - 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x62, - 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x67, 0x65, 0x6e, 0x2f, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, } var ( @@ -5248,7 +5166,7 @@ var file_FlightSql_proto_goTypes = []interface{}{ (*DoPutUpdateResult)(nil), // 52: arrow.flight.protocol.sql.DoPutUpdateResult (*ActionCancelQueryRequest)(nil), // 53: arrow.flight.protocol.sql.ActionCancelQueryRequest (*ActionCancelQueryResult)(nil), // 54: arrow.flight.protocol.sql.ActionCancelQueryResult - (*descriptor.MessageOptions)(nil), // 55: google.protobuf.MessageOptions + (*descriptorpb.MessageOptions)(nil), // 55: google.protobuf.MessageOptions } var file_FlightSql_proto_depIdxs = []int32{ 36, // 0: arrow.flight.protocol.sql.ActionCreatePreparedSubstraitPlanRequest.plan:type_name -> arrow.flight.protocol.sql.SubstraitPlan diff --git a/go/arrow/flight/gen/flight/Flight_grpc.pb.go b/go/arrow/flight/gen/flight/Flight_grpc.pb.go index 9613114448796..150de71a1113b 100644 --- a/go/arrow/flight/gen/flight/Flight_grpc.pb.go +++ b/go/arrow/flight/gen/flight/Flight_grpc.pb.go @@ -1,8 +1,4 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. -// versions: -// - protoc-gen-go-grpc v1.2.0 -// - protoc v3.12.4 -// source: Flight.proto package flight @@ -15,20 +11,17 @@ 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 @@ -36,7 +29,6 @@ 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 @@ -48,19 +40,39 @@ 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 + // should return as quickly as possible. (GetFlightInfo doesn't + // return until the query is complete.) + // + // A client can consume any available results before + // the query is completed. See PollInfo.info for details. + // + // A client can poll the updated query status by calling + // PollFlightInfo() with PollInfo.flight_descriptor. A server + // should not respond until the result would be different from last + // time. That way, the client can "long poll" for updates + // without constantly making requests. Clients can set a short timeout + // to avoid blocking calls if desired. + // + // A client can't use PollInfo.flight_descriptor after + // PollInfo.expiration_time passes. A server might not accept the + // retry descriptor anymore and the query may be cancelled. // + // 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 @@ -68,14 +80,12 @@ 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 @@ -83,7 +93,6 @@ 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. @@ -99,7 +108,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 } @@ -130,7 +139,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 } @@ -170,6 +179,15 @@ func (c *flightServiceClient) GetFlightInfo(ctx context.Context, in *FlightDescr return out, nil } +func (c *flightServiceClient) PollFlightInfo(ctx context.Context, in *FlightDescriptor, opts ...grpc.CallOption) (*PollInfo, error) { + out := new(PollInfo) + err := c.cc.Invoke(ctx, "/arrow.flight.protocol.FlightService/PollFlightInfo", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *flightServiceClient) GetSchema(ctx context.Context, in *FlightDescriptor, opts ...grpc.CallOption) (*SchemaResult, error) { out := new(SchemaResult) err := c.cc.Invoke(ctx, "/arrow.flight.protocol.FlightService/GetSchema", in, out, opts...) @@ -180,7 +198,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 } @@ -212,7 +230,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 } @@ -243,7 +261,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 } @@ -274,7 +292,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 } @@ -306,7 +324,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 } @@ -341,13 +359,11 @@ 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 @@ -355,7 +371,6 @@ 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 @@ -367,19 +382,39 @@ 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 + // should return as quickly as possible. (GetFlightInfo doesn't + // return until the query is complete.) + // + // A client can consume any available results before + // the query is completed. See PollInfo.info for details. + // + // A client can poll the updated query status by calling + // PollFlightInfo() with PollInfo.flight_descriptor. A server + // should not respond until the result would be different from last + // time. That way, the client can "long poll" for updates + // without constantly making requests. Clients can set a short timeout + // to avoid blocking calls if desired. + // + // A client can't use PollInfo.flight_descriptor after + // PollInfo.expiration_time passes. A server might not accept the + // retry descriptor anymore and the query may be cancelled. // + // 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 @@ -387,14 +422,12 @@ 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 @@ -402,7 +435,6 @@ 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. @@ -423,6 +455,9 @@ func (UnimplementedFlightServiceServer) ListFlights(*Criteria, FlightService_Lis func (UnimplementedFlightServiceServer) GetFlightInfo(context.Context, *FlightDescriptor) (*FlightInfo, error) { return nil, status.Errorf(codes.Unimplemented, "method GetFlightInfo not implemented") } +func (UnimplementedFlightServiceServer) PollFlightInfo(context.Context, *FlightDescriptor) (*PollInfo, error) { + return nil, status.Errorf(codes.Unimplemented, "method PollFlightInfo not implemented") +} func (UnimplementedFlightServiceServer) GetSchema(context.Context, *FlightDescriptor) (*SchemaResult, error) { return nil, status.Errorf(codes.Unimplemented, "method GetSchema not implemented") } @@ -450,8 +485,8 @@ type UnsafeFlightServiceServer interface { mustEmbedUnimplementedFlightServiceServer() } -func RegisterFlightServiceServer(s grpc.ServiceRegistrar, srv FlightServiceServer) { - s.RegisterService(&FlightService_ServiceDesc, srv) +func RegisterFlightServiceServer(s *grpc.Server, srv FlightServiceServer) { + s.RegisterService(&_FlightService_serviceDesc, srv) } func _FlightService_Handshake_Handler(srv interface{}, stream grpc.ServerStream) error { @@ -519,6 +554,24 @@ func _FlightService_GetFlightInfo_Handler(srv interface{}, ctx context.Context, return interceptor(ctx, in, info, handler) } +func _FlightService_PollFlightInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(FlightDescriptor) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(FlightServiceServer).PollFlightInfo(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/arrow.flight.protocol.FlightService/PollFlightInfo", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(FlightServiceServer).PollFlightInfo(ctx, req.(*FlightDescriptor)) + } + return interceptor(ctx, in, info, handler) +} + func _FlightService_GetSchema_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(FlightDescriptor) if err := dec(in); err != nil { @@ -652,10 +705,7 @@ func (x *flightServiceListActionsServer) Send(m *ActionType) error { return x.ServerStream.SendMsg(m) } -// 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{ +var _FlightService_serviceDesc = grpc.ServiceDesc{ ServiceName: "arrow.flight.protocol.FlightService", HandlerType: (*FlightServiceServer)(nil), Methods: []grpc.MethodDesc{ @@ -663,6 +713,10 @@ var FlightService_ServiceDesc = grpc.ServiceDesc{ MethodName: "GetFlightInfo", Handler: _FlightService_GetFlightInfo_Handler, }, + { + MethodName: "PollFlightInfo", + Handler: _FlightService_PollFlightInfo_Handler, + }, { MethodName: "GetSchema", Handler: _FlightService_GetSchema_Handler, diff --git a/go/arrow/flight/server.go b/go/arrow/flight/server.go index c5e64986d5f78..88006b2fe2556 100644 --- a/go/arrow/flight/server.go +++ b/go/arrow/flight/server.go @@ -42,6 +42,7 @@ type ( FlightEndpoint = flight.FlightEndpoint Location = flight.Location FlightInfo = flight.FlightInfo + PollInfo = flight.PollInfo FlightData = flight.FlightData PutResult = flight.PutResult Ticket = flight.Ticket diff --git a/go/arrow/internal/flight_integration/scenario.go b/go/arrow/internal/flight_integration/scenario.go index 3e5cc73b98753..6646502ab648f 100644 --- a/go/arrow/internal/flight_integration/scenario.go +++ b/go/arrow/internal/flight_integration/scenario.go @@ -22,6 +22,7 @@ import ( "errors" "fmt" "io" + "math" "net" "os" "reflect" @@ -68,6 +69,8 @@ func GetScenario(name string, args ...string) Scenario { return &expirationTimeCancelFlightInfoScenarioTester{} case "expiration_time:renew_flight_endpoint": return &expirationTimeRenewFlightEndpointScenarioTester{} + case "poll_flight_info": + return &pollFlightInfoScenarioTester{} case "flight_sql": return &flightSqlScenarioTester{} case "flight_sql:extension": @@ -1131,6 +1134,108 @@ func (tester *expirationTimeRenewFlightEndpointScenarioTester) RunClient(addr st return nil } +type pollFlightInfoScenarioTester struct { + flight.BaseFlightServer +} + +func (tester *pollFlightInfoScenarioTester) MakeServer(port int) flight.Server { + srv := flight.NewServerWithMiddleware(nil) + srv.RegisterFlightService(tester) + initServer(port, srv) + return srv +} + +func (tester *pollFlightInfoScenarioTester) PollFlightInfo(ctx context.Context, desc *flight.FlightDescriptor) (*flight.PollInfo, error) { + schema := arrow.NewSchema( + []arrow.Field{ + {Name: "number", Type: arrow.PrimitiveTypes.Uint32}, + }, + nil, + ) + endpoints := []*flight.FlightEndpoint{ + &flight.FlightEndpoint{ + Ticket: &flight.Ticket{Ticket: []byte("long-running query")}, + Location: []*flight.Location{}, + }, + } + info := &flight.FlightInfo{ + Schema: flight.SerializeSchema(schema, memory.DefaultAllocator), + FlightDescriptor: desc, + Endpoint: endpoints, + TotalRecords: -1, + TotalBytes: -1, + } + pollDesc := flight.FlightDescriptor{ + Type: flight.DescriptorCMD, + Cmd: []byte("poll"), + } + if desc.Type == pollDesc.Type && string(desc.Cmd) == string(pollDesc.Cmd) { + progress := float64(1.0) + return &flight.PollInfo{ + Info: info, + FlightDescriptor: nil, + Progress: &progress, + ExpirationTime: nil, + }, nil + } else { + progress := float64(0.1) + return &flight.PollInfo{ + Info: info, + FlightDescriptor: &pollDesc, + Progress: &progress, + ExpirationTime: timestamppb.New(time.Now().Add(time.Second * 10)), + }, nil + } +} + +func (tester *pollFlightInfoScenarioTester) 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("heavy query"), + } + info, err := client.PollFlightInfo(ctx, &desc) + if err != nil { + return err + } + switch { + case info.FlightDescriptor == nil: + return fmt.Errorf("description is missing: %s", info.String()) + case info.Progress == nil: + return fmt.Errorf("progress is missing: %s", info.String()) + case !(0.0 <= *info.Progress && *info.Progress <= 1.0): + return fmt.Errorf("invalid progress: %s", info.String()) + case info.ExpirationTime == nil: + return fmt.Errorf("expiration time is missing: %s", info.String()) + } + + info, err = client.PollFlightInfo(ctx, info.FlightDescriptor) + if err != nil { + return err + } + switch { + case info.FlightDescriptor != nil: + return fmt.Errorf("retried but no finished yet: %s", info.String()) + case info.Progress == nil: + return fmt.Errorf("progress is missing in finished query: %s", + info.String()) + case math.Abs(*info.Progress-1.0) > 1e-5: + return fmt.Errorf("progress for finished query isn't 1.0: %s", + info.String()) + case info.ExpirationTime != nil: + return fmt.Errorf("expiration time must not be set for finished query: %s", + info.String()) + } + + return nil +} + const ( updateStatementExpectedRows int64 = 10000 updateStatementWithTransactionExpectedRows int64 = 15000 diff --git a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java index 155e373bda2f3..422ed01c394d1 100644 --- a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java +++ b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java @@ -287,6 +287,23 @@ public FlightInfo getInfo(FlightDescriptor descriptor, CallOption... options) { } } + /** + * Start or get info on execution of a long-running query. + * + * @param descriptor The descriptor for the stream. + * @param options RPC-layer hints for this call. + * @return Metadata about execution. + */ + public PollInfo pollInfo(FlightDescriptor descriptor, CallOption... options) { + try { + return new PollInfo(CallOptions.wrapStub(blockingStub, options).pollFlightInfo(descriptor.toProtocol())); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } catch (StatusRuntimeException sre) { + throw StatusUtils.fromGrpcRuntimeException(sre); + } + } + /** * Get schema for a stream. * @param descriptor The descriptor for the stream. diff --git a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightMethod.java b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightMethod.java index 5d2915bb686c4..b1331a8edd60f 100644 --- a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightMethod.java +++ b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightMethod.java @@ -32,6 +32,7 @@ public enum FlightMethod { DO_ACTION, LIST_ACTIONS, DO_EXCHANGE, + POLL_FLIGHT_INFO, ; /** @@ -58,6 +59,8 @@ public static FlightMethod fromProtocol(final String methodName) { return LIST_ACTIONS; } else if (FlightServiceGrpc.getDoExchangeMethod().getFullMethodName().equals(methodName)) { return DO_EXCHANGE; + } else if (FlightServiceGrpc.getPollFlightInfoMethod().getFullMethodName().equals(methodName)) { + return POLL_FLIGHT_INFO; } throw new IllegalArgumentException("Not a Flight method name in gRPC: " + methodName); } diff --git a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightProducer.java b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightProducer.java index 5e5b265050086..f2ae3db0b50d7 100644 --- a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightProducer.java +++ b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightProducer.java @@ -52,6 +52,22 @@ void listFlights(CallContext context, Criteria criteria, */ FlightInfo getFlightInfo(CallContext context, FlightDescriptor descriptor); + /** + * Begin or get an update on execution of a long-running query. + * + * <p>If the descriptor would begin a query, the server should return a response immediately to not + * block the client. Otherwise, the server should not return an update until progress is made to + * not spam the client with inactionable updates. + * + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about execution. + */ + default PollInfo pollFlightInfo(CallContext context, FlightDescriptor descriptor) { + FlightInfo info = getFlightInfo(context, descriptor); + return new PollInfo(info, null, null, null); + } + /** * Get schema for a particular data stream. * diff --git a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightService.java b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightService.java index 29a4f2bbd19ea..5231a7aaf76e4 100644 --- a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightService.java +++ b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightService.java @@ -258,6 +258,21 @@ public void getFlightInfo(Flight.FlightDescriptor request, StreamObserver<Flight responseObserver.onCompleted(); } + @Override + public void pollFlightInfo(Flight.FlightDescriptor request, StreamObserver<Flight.PollInfo> responseObserver) { + final PollInfo info; + try { + info = producer + .pollFlightInfo(makeContext((ServerCallStreamObserver<?>) responseObserver), new FlightDescriptor(request)); + } catch (Exception ex) { + // Don't capture exceptions from onNext or onCompleted with this block - because then we can't call onError + responseObserver.onError(StatusUtils.toGrpcException(ex)); + return; + } + responseObserver.onNext(info.toProtocol()); + responseObserver.onCompleted(); + } + /** * Broadcast the given exception to all registered middleware. */ diff --git a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/PollInfo.java b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/PollInfo.java new file mode 100644 index 0000000000000..2bb3c6db69569 --- /dev/null +++ b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/PollInfo.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.arrow.flight; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.util.Objects; +import java.util.Optional; + +import org.apache.arrow.flight.impl.Flight; + +import com.google.protobuf.Timestamp; + +/** + * A POJO representation of the execution of a long-running query. + */ +public class PollInfo { + private final FlightInfo flightInfo; + private final FlightDescriptor flightDescriptor; + private final Double progress; + private final Instant expirationTime; + + /** + * Create a new PollInfo. + * + * @param flightInfo The FlightInfo (must not be null). + * @param flightDescriptor The descriptor used to poll for more information; null if and only if query is finished. + * @param progress Optional progress info in [0.0, 1.0]. + * @param expirationTime An expiration time, after which the server may no longer recognize the descriptor. + */ + public PollInfo(FlightInfo flightInfo, FlightDescriptor flightDescriptor, Double progress, Instant expirationTime) { + this.flightInfo = Objects.requireNonNull(flightInfo); + this.flightDescriptor = flightDescriptor; + this.progress = progress; + this.expirationTime = expirationTime; + } + + PollInfo(Flight.PollInfo flt) throws URISyntaxException { + this.flightInfo = new FlightInfo(flt.getInfo()); + this.flightDescriptor = flt.hasFlightDescriptor() ? new FlightDescriptor(flt.getFlightDescriptor()) : null; + this.progress = flt.hasProgress() ? flt.getProgress() : null; + this.expirationTime = flt.hasExpirationTime() ? + Instant.ofEpochSecond(flt.getExpirationTime().getSeconds(), flt.getExpirationTime().getNanos()) : + null; + } + + /** + * The FlightInfo describing the result set of the execution of a query. + * + * <p>This is always present and always contains all endpoints for the query execution so far, + * not just new endpoints that completed execution since the last call to + * {@link FlightClient#pollInfo(FlightDescriptor, CallOption...)}. + */ + public FlightInfo getFlightInfo() { + return flightInfo; + } + + /** + * The FlightDescriptor that should be used to get further updates on this query. + * + * <p>It is present if and only if the query is still running. If present, it should be passed to + * {@link FlightClient#pollInfo(FlightDescriptor, CallOption...)} to get an update. + */ + public Optional<FlightDescriptor> getFlightDescriptor() { + return Optional.ofNullable(flightDescriptor); + } + + /** + * The progress of the query. + * + * <p>If present, should be a value in [0.0, 1.0]. It is not necessarily monotonic or non-decreasing. + */ + public Optional<Double> getProgress() { + return Optional.ofNullable(progress); + } + + /** + * The expiration time of the query execution. + * + * <p>After this passes, the server may not recognize the descriptor anymore and the client will not + * be able to track the query anymore. + */ + public Optional<Instant> getExpirationTime() { + return Optional.ofNullable(expirationTime); + } + + Flight.PollInfo toProtocol() { + Flight.PollInfo.Builder b = Flight.PollInfo.newBuilder(); + b.setInfo(flightInfo.toProtocol()); + if (flightDescriptor != null) { + b.setFlightDescriptor(flightDescriptor.toProtocol()); + } + if (progress != null) { + b.setProgress(progress); + } + if (expirationTime != null) { + b.setExpirationTime( + Timestamp.newBuilder() + .setSeconds(expirationTime.getEpochSecond()) + .setNanos(expirationTime.getNano()) + .build()); + } + return b.build(); + } + + public ByteBuffer serialize() { + return ByteBuffer.wrap(toProtocol().toByteArray()); + } + + public static PollInfo deserialize(ByteBuffer serialized) throws IOException, URISyntaxException { + return new PollInfo(Flight.PollInfo.parseFrom(serialized)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PollInfo pollInfo = (PollInfo) o; + return Objects.equals(getFlightInfo(), pollInfo.getFlightInfo()) && + Objects.equals(getFlightDescriptor(), pollInfo.getFlightDescriptor()) && + Objects.equals(getProgress(), pollInfo.getProgress()) && + Objects.equals(getExpirationTime(), pollInfo.getExpirationTime()); + } + + @Override + public int hashCode() { + return Objects.hash(getFlightInfo(), getFlightDescriptor(), getProgress(), getExpirationTime()); + } + + @Override + public String toString() { + return "PollInfo{" + + "flightInfo=" + flightInfo + + ", flightDescriptor=" + flightDescriptor + + ", progress=" + progress + + ", expirationTime=" + expirationTime + + '}'; + } +} diff --git a/java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/PollFlightInfoProducer.java b/java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/PollFlightInfoProducer.java new file mode 100644 index 0000000000000..61312db56bc61 --- /dev/null +++ b/java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/PollFlightInfoProducer.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.arrow.flight.integration.tests; + +import java.nio.charset.StandardCharsets; +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import org.apache.arrow.flight.FlightDescriptor; +import org.apache.arrow.flight.FlightEndpoint; +import org.apache.arrow.flight.FlightInfo; +import org.apache.arrow.flight.NoOpFlightProducer; +import org.apache.arrow.flight.PollInfo; +import org.apache.arrow.flight.Ticket; +import org.apache.arrow.vector.types.Types; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.Schema; + +/** Test PollFlightInfo. */ +class PollFlightInfoProducer extends NoOpFlightProducer { + static final byte[] POLL_DESCRIPTOR = "poll".getBytes(StandardCharsets.UTF_8); + + @Override + public PollInfo pollFlightInfo(CallContext context, FlightDescriptor descriptor) { + Schema schema = new Schema( + Collections.singletonList(Field.notNullable("number", Types.MinorType.UINT4.getType()))); + List<FlightEndpoint> endpoints = Collections.singletonList( + new FlightEndpoint( + new Ticket("long-running query".getBytes(StandardCharsets.UTF_8)))); + FlightInfo info = new FlightInfo(schema, descriptor, endpoints, -1, -1 ); + if (descriptor.isCommand() && Arrays.equals(descriptor.getCommand(), POLL_DESCRIPTOR)) { + return new PollInfo(info, null, 1.0, null); + } else { + return new PollInfo( + info, FlightDescriptor.command(POLL_DESCRIPTOR), 0.1, Instant.now().plus(10, ChronoUnit.SECONDS)); + } + } +} diff --git a/java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/PollFlightInfoScenario.java b/java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/PollFlightInfoScenario.java new file mode 100644 index 0000000000000..1ffbe3c726216 --- /dev/null +++ b/java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/PollFlightInfoScenario.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.arrow.flight.integration.tests; + +import java.nio.charset.StandardCharsets; +import java.util.Optional; + +import org.apache.arrow.flight.FlightClient; +import org.apache.arrow.flight.FlightDescriptor; +import org.apache.arrow.flight.FlightProducer; +import org.apache.arrow.flight.FlightServer; +import org.apache.arrow.flight.Location; +import org.apache.arrow.flight.PollInfo; +import org.apache.arrow.memory.BufferAllocator; + +/** Test PollFlightInfo. */ +final class PollFlightInfoScenario implements Scenario { + @Override + public FlightProducer producer(BufferAllocator allocator, Location location) throws Exception { + return new PollFlightInfoProducer(); + } + + @Override + public void buildServer(FlightServer.Builder builder) throws Exception { + } + + @Override + public void client(BufferAllocator allocator, Location location, FlightClient client) throws Exception { + PollInfo info = client.pollInfo(FlightDescriptor.command("heavy query".getBytes(StandardCharsets.UTF_8))); + IntegrationAssertions.assertNotNull(info.getFlightInfo()); + Optional<Double> progress = info.getProgress(); + IntegrationAssertions.assertTrue("progress is missing", progress.isPresent()); + IntegrationAssertions.assertTrue("progress is invalid", progress.get() >= 0.0 && progress.get() <= 1.0); + IntegrationAssertions.assertTrue("expiration is missing", info.getExpirationTime().isPresent()); + IntegrationAssertions.assertTrue("descriptor is missing", + info.getFlightDescriptor().isPresent()); + + info = client.pollInfo(info.getFlightDescriptor().get()); + IntegrationAssertions.assertNotNull(info.getFlightInfo()); + progress = info.getProgress(); + IntegrationAssertions.assertTrue("progress is missing in finished query", progress.isPresent()); + IntegrationAssertions.assertTrue("progress isn't 1.0 in finished query", + Math.abs(progress.get() - 1.0) < Math.ulp(1.0)); + IntegrationAssertions.assertFalse("expiration is set in finished query", info.getExpirationTime().isPresent()); + IntegrationAssertions.assertFalse("descriptor is set in finished query", info.getFlightDescriptor().isPresent()); + } +} diff --git a/java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/Scenarios.java b/java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/Scenarios.java index da9064c0e9359..26629c650e30f 100644 --- a/java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/Scenarios.java +++ b/java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/Scenarios.java @@ -46,6 +46,7 @@ private Scenarios() { scenarios.put("expiration_time:list_actions", ExpirationTimeListActionsScenario::new); scenarios.put("middleware", MiddlewareScenario::new); scenarios.put("ordered", OrderedScenario::new); + scenarios.put("poll_flight_info", PollFlightInfoScenario::new); scenarios.put("flight_sql", FlightSqlScenario::new); scenarios.put("flight_sql:extension", FlightSqlExtensionScenario::new); } diff --git a/java/flight/flight-integration-tests/src/test/java/org/apache/arrow/flight/integration/tests/IntegrationTest.java b/java/flight/flight-integration-tests/src/test/java/org/apache/arrow/flight/integration/tests/IntegrationTest.java index ab7f04075eecc..cf65e16fac06f 100644 --- a/java/flight/flight-integration-tests/src/test/java/org/apache/arrow/flight/integration/tests/IntegrationTest.java +++ b/java/flight/flight-integration-tests/src/test/java/org/apache/arrow/flight/integration/tests/IntegrationTest.java @@ -63,6 +63,11 @@ void ordered() throws Exception { testScenario("ordered"); } + @Test + void pollFlightInfo() throws Exception { + testScenario("poll_flight_info"); + } + @Test void flightSql() throws Exception { testScenario("flight_sql");