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/18 18:02:27 UTC

[GitHub] [arrow] lidavidm commented on a change in pull request #12669: ARROW-15974: Migrate flight/types.h header definitions to use Result<>

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



##########
File path: cpp/src/arrow/flight/types.cc
##########
@@ -337,23 +366,40 @@ bool ActionType::Equals(const ActionType& other) const {
   return type == other.type && description == other.description;
 }
 
-Status MetadataRecordBatchReader::ReadAll(
-    std::vector<std::shared_ptr<RecordBatch>>* batches) {
-  FlightStreamChunk chunk;
+Status ResultStream::Next(std::unique_ptr<Result>* info) {
+  return Next().Value(info);
+}
+
+Status MetadataRecordBatchReader::Next(FlightStreamChunk* next) {
+  ARROW_ASSIGN_OR_RAISE(*next, Next());
+  return Status::OK();

Review comment:
       nit: `return Next().Value(next);` should work too?

##########
File path: cpp/src/arrow/python/flight.cc
##########
@@ -206,11 +206,13 @@ PyFlightResultStream::PyFlightResultStream(PyObject* generator,
   generator_.reset(generator);
 }
 
-Status PyFlightResultStream::Next(std::unique_ptr<arrow::flight::Result>* result) {
-  return SafeCallIntoPython([=] {
-    const Status status = callback_(generator_.obj(), result);
+arrow::Result<std::unique_ptr<arrow::flight::Result>> PyFlightResultStream::Next() {
+  return SafeCallIntoPython(
+      [=]() -> arrow::Result<std::unique_ptr<arrow::flight::Result>> {
+    std::unique_ptr<arrow::flight::Result> result;
+    const Status status = callback_(generator_.obj(), &result);
     RETURN_NOT_OK(CheckPyError());

Review comment:
       Needs RETURN_NOT_OK(status)

##########
File path: cpp/src/arrow/flight/types.h
##########
@@ -507,14 +533,26 @@ class ARROW_FLIGHT_EXPORT MetadataRecordBatchReader {
 
   /// \brief Get the schema for this stream.
   virtual arrow::Result<std::shared_ptr<Schema>> GetSchema() = 0;
+
   /// \brief Get the next message from Flight. If the stream is
   /// finished, then the members of \a FlightStreamChunk will be
   /// nullptr.
-  virtual Status Next(FlightStreamChunk* next) = 0;
+  virtual arrow::Result<FlightStreamChunk> Next() = 0;
+
+  ARROW_DEPRECATED("Deprecated in 8.0.0. Use Result-returning overload instead.")
+  Status Next(FlightStreamChunk* next);
+
   /// \brief Consume entire stream as a vector of record batches
-  virtual Status ReadAll(std::vector<std::shared_ptr<RecordBatch>>* batches);
+  virtual arrow::Result<std::vector<std::shared_ptr<RecordBatch>>> ToRecordBatches();
+
+  ARROW_DEPRECATED("Deprecated in 8.0.0. Use ToRecordBatches overload instead.")
+  Status ReadAll(std::vector<std::shared_ptr<RecordBatch>>* batches);
+
   /// \brief Consume entire stream as a Table
-  virtual Status ReadAll(std::shared_ptr<Table>* table);
+  virtual arrow::Result<std::shared_ptr<Table>> ToTable();
+
+  ARROW_DEPRECATED("Deprecated in 8.0.0. Use ToTable overload instead.")
+  Status ReadAll(std::shared_ptr<Table>* table);

Review comment:
       These are analogous to these APIs:
   https://github.com/apache/arrow/blob/d45931160baa5618659edaedaa7b2ea5885ce678/cpp/src/arrow/record_batch.h#L299-L303
   
   If we want to deprecate these, we should deprecate those as well I think




-- 
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