You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/03/31 13:00:28 UTC

[GitHub] [arrow] lidavidm commented on a change in pull request #12719: ARROW-16032: [C++] Migrate FlightClient API to Result<>

lidavidm commented on a change in pull request #12719:
URL: https://github.com/apache/arrow/pull/12719#discussion_r839569118



##########
File path: cpp/src/arrow/flight/flight_benchmark.cc
##########
@@ -124,7 +124,7 @@ Status WaitForReady(FlightClient* client, const FlightCallOptions& call_options)
   Action action{"ping", nullptr};
   for (int attempt = 0; attempt < 10; attempt++) {
     std::unique_ptr<ResultStream> stream;
-    if (client->DoAction(call_options, action, &stream).ok()) {
+    if (client->DoAction(call_options, action).Value(&stream).ok()) {

Review comment:
       Just use `Result::ok`, you can remove `stream` and `Value()`

##########
File path: cpp/src/arrow/flight/integration_tests/test_integration_client.cc
##########
@@ -94,7 +94,7 @@ Status ConsumeFlightLocation(
     FlightClient* read_client, const Ticket& ticket,
     const std::vector<std::shared_ptr<RecordBatch>>& retrieved_data) {
   std::unique_ptr<FlightStreamReader> stream;
-  RETURN_NOT_OK(read_client->DoGet(ticket, &stream));
+  RETURN_NOT_OK(read_client->DoGet(ticket).Value(&stream));

Review comment:
       use ARROW_ASSIGN_OR_RAISE

##########
File path: cpp/src/arrow/flight/test_definitions.cc
##########
@@ -156,10 +154,10 @@ void DataTest::CheckDoGet(const Ticket& ticket,
   ASSERT_GE(num_batches, 2);
 
   std::unique_ptr<FlightStreamReader> stream;
-  ASSERT_OK(client_->DoGet(ticket, &stream));
+  ASSERT_OK_AND_ASSIGN(stream, client_->DoGet(ticket));

Review comment:
       and ditto below

##########
File path: cpp/src/arrow/flight/test_definitions.cc
##########
@@ -131,15 +130,14 @@ void DataTest::TearDown() {
 Status DataTest::ConnectClient() {
   ARROW_ASSIGN_OR_RAISE(auto location,
                         Location::ForScheme(transport(), "localhost", server_->port()));
-  return FlightClient::Connect(location, &client_);
+  return FlightClient::Connect(location).Value(&client_);

Review comment:
       ```suggestion
     ARROW_ASSIGN_OR_RAISE(client_, FlightClient::Connect(location));
     return Status::OK();
   ```

##########
File path: cpp/src/arrow/flight/sql/client.h
##########
@@ -185,17 +182,20 @@ class ARROW_EXPORT FlightSqlClient {
                        const std::shared_ptr<Schema>& schema,
                        std::unique_ptr<FlightStreamWriter>* stream,
                        std::unique_ptr<FlightMetadataReader>* reader) {
-    return impl_->DoPut(options, descriptor, schema, stream, reader);
+    ARROW_ASSIGN_OR_RAISE(auto result, impl_->DoPut(options, descriptor, schema));

Review comment:
       (I think these are declared virtual only so we can do mocking/testing)

##########
File path: cpp/src/arrow/flight/sql/client.h
##########
@@ -185,17 +182,20 @@ class ARROW_EXPORT FlightSqlClient {
                        const std::shared_ptr<Schema>& schema,
                        std::unique_ptr<FlightStreamWriter>* stream,
                        std::unique_ptr<FlightMetadataReader>* reader) {
-    return impl_->DoPut(options, descriptor, schema, stream, reader);
+    ARROW_ASSIGN_OR_RAISE(auto result, impl_->DoPut(options, descriptor, schema));

Review comment:
       Can we create a ticket to refactor the Flight SQL APIs as well

##########
File path: c_glib/arrow-flight-glib/client.cpp
##########
@@ -252,11 +252,11 @@ gaflight_client_new(GAFlightLocation *location,
   if (options) {
     const auto flight_options = gaflight_client_options_get_raw(options);
     status = arrow::flight::FlightClient::Connect(*flight_location,
-                                                  *flight_options,
-                                                  &flight_client);
+                                                  *flight_options
+                                          ).Value(&flight_client);
   } else {
-    status = arrow::flight::FlightClient::Connect(*flight_location,
-                                                  &flight_client);
+    status = arrow::flight::FlightClient::Connect(*flight_location
+                                                  ).Value(&flight_client);

Review comment:
       nit, but it might be good to break up some of these into multiple lines, e.g.
   
   ```
   auto result = arrow::flight::FlightClient::Connect(...); // with original indentation
   status = result.Value(&flight_client);
   ```
   just because the indentation here is starting to get awkward

##########
File path: cpp/src/arrow/flight/client.h
##########
@@ -283,30 +309,49 @@ class ARROW_FLIGHT_EXPORT FlightClient {
   }
 
   /// \brief List all available flights known to the server
-  /// \param[out] listing an iterator that returns a FlightInfo for each flight
-  /// \return Status
+  /// \return Arrow result with an iterator that returns a FlightInfo for each flight
+  arrow::Result<std::unique_ptr<FlightListing>> ListFlights();
+
+  ARROW_DEPRECATED("Deprecated in 8.0.0. Use Result-returning overload instead.")
   Status ListFlights(std::unique_ptr<FlightListing>* listing);
 
   /// \brief List available flights given indicated filter criteria
   /// \param[in] options Per-RPC options
   /// \param[in] criteria the filter criteria (opaque)
-  /// \param[out] listing an iterator that returns a FlightInfo for each flight
-  /// \return Status
+  /// \return Arrow result with an iterator that returns a FlightInfo for each flight
+  arrow::Result<std::unique_ptr<FlightListing>> ListFlights(
+      const FlightCallOptions& options, const Criteria& criteria);
+
+  ARROW_DEPRECATED("Deprecated in 8.0.0. Use Result-returning overload instead.")
   Status ListFlights(const FlightCallOptions& options, const Criteria& criteria,
                      std::unique_ptr<FlightListing>* listing);
 
   /// \brief Given a flight ticket and schema, request to be sent the
   /// stream. Returns record batch stream reader
   /// \param[in] options Per-RPC options
   /// \param[in] ticket The flight ticket to use
-  /// \param[out] stream the returned RecordBatchReader
-  /// \return Status
+  /// \return Arrow result with the returned RecordBatchReader
+  arrow::Result<std::unique_ptr<FlightStreamReader>> DoGet(
+      const FlightCallOptions& options, const Ticket& ticket);
+  arrow::Result<std::unique_ptr<FlightStreamReader>> DoGet(const Ticket& ticket) {
+    return DoGet({}, ticket);
+  }
+
+  ARROW_DEPRECATED("Deprecated in 8.0.0. Use Result-returning overload instead.")
   Status DoGet(const FlightCallOptions& options, const Ticket& ticket,
                std::unique_ptr<FlightStreamReader>* stream);
+  ARROW_DEPRECATED("Deprecated in 8.0.0. Use Result-returning overload instead.")
   Status DoGet(const Ticket& ticket, std::unique_ptr<FlightStreamReader>* stream) {
-    return DoGet({}, ticket, stream);
+    return DoGet({}, ticket).Value(stream);
   }
 
+  /// \brief DoPut return value
+  struct DoPutResult {
+    /// \brief a writer to write record batches to
+    std::unique_ptr<FlightStreamWriter> stream;

Review comment:
       nit: `writer`?

##########
File path: cpp/src/arrow/flight/sql/server_test.cc
##########
@@ -164,7 +164,7 @@ class TestFlightSqlServer : public ::testing::Test {
 
     std::unique_ptr<FlightClient> client;
     ASSERT_OK_AND_ASSIGN(auto location, Location::Parse(uri));
-    ASSERT_OK(FlightClient::Connect(location, &client));
+    ASSERT_OK_AND_ASSIGN(client, FlightClient::Connect(location));

Review comment:
       ```suggestion
       ASSERT_OK_AND_ASSIGN(auto client, FlightClient::Connect(location));
   ```

##########
File path: cpp/src/arrow/flight/test_definitions.cc
##########
@@ -156,10 +154,10 @@ void DataTest::CheckDoGet(const Ticket& ticket,
   ASSERT_GE(num_batches, 2);
 
   std::unique_ptr<FlightStreamReader> stream;
-  ASSERT_OK(client_->DoGet(ticket, &stream));
+  ASSERT_OK_AND_ASSIGN(stream, client_->DoGet(ticket));

Review comment:
       ```suggestion
     ASSERT_OK_AND_ASSIGN(auto stream, client_->DoGet(ticket));
   ```

##########
File path: cpp/src/arrow/flight/sql/test_app_cli.cc
##########
@@ -103,7 +103,7 @@ Status PrintResults(FlightSqlClient& client, const FlightCallOptions& call_optio
 Status RunMain() {
   std::unique_ptr<FlightClient> client;
   ARROW_ASSIGN_OR_RAISE(auto location, Location::ForGrpcTcp(FLAGS_host, FLAGS_port));
-  ARROW_RETURN_NOT_OK(FlightClient::Connect(location, &client));
+  ARROW_ASSIGN_OR_RAISE(client, FlightClient::Connect(location));

Review comment:
       ```suggestion
     ARROW_ASSIGN_OR_RAISE(auto client, FlightClient::Connect(location));
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org