pitrou commented on a change in pull request #9842:
URL: https://github.com/apache/arrow/pull/9842#discussion_r607706980



##########
File path: cpp/src/arrow/filesystem/s3fs.cc
##########
@@ -1142,82 +1145,67 @@ struct TreeWalker : public 
std::enable_shared_from_this<TreeWalker> {
         recursion_handler_(std::move(recursion_handler)) {}
 
  private:
-  std::mutex mutex_;
-  Future<> future_;
-  std::atomic<int32_t> num_in_flight_;
+  std::shared_ptr<TaskGroup> task_group_;
+  util::Mutex mutex_;

Review comment:
       Is there a reason not to use `std::mutex` here?

##########
File path: cpp/src/arrow/filesystem/s3fs.cc
##########
@@ -1142,82 +1145,67 @@ struct TreeWalker : public 
std::enable_shared_from_this<TreeWalker> {
         recursion_handler_(std::move(recursion_handler)) {}
 
  private:
-  std::mutex mutex_;
-  Future<> future_;
-  std::atomic<int32_t> num_in_flight_;
+  std::shared_ptr<TaskGroup> task_group_;
+  util::Mutex mutex_;
 
   Status DoWalk() {
-    future_ = decltype(future_)::Make();
-    num_in_flight_ = 0;
+    task_group_ =
+        TaskGroup::MakeThreaded(io_context_.executor(), 
io_context_.stop_token());
     WalkChild(base_dir_, /*nesting_depth=*/0);
     // When this returns, ListObjectsV2 tasks either have finished or will 
exit early
-    return future_.status();
+    return task_group_->Finish();
   }
 
-  bool is_finished() const { return future_.is_finished(); }
-
-  void ListObjectsFinished(Status st) {
-    const auto in_flight = --num_in_flight_;
-    if (!st.ok() || !in_flight) {
-      future_.MarkFinished(std::move(st));
-    }
-  }
+  bool ok() const { return task_group_->ok(); }
 
   struct ListObjectsV2Handler {
     std::shared_ptr<TreeWalker> walker;
     std::string prefix;
     int32_t nesting_depth;
     S3Model::ListObjectsV2Request req;
 
-    void operator()(const Result<S3Model::ListObjectsV2Outcome>& result) {
+    Status operator()(const Result<S3Model::ListObjectsV2Outcome>& result) {
       // Serialize calls to operation-specific handlers
-      std::unique_lock<std::mutex> guard(walker->mutex_);
-      if (walker->is_finished()) {
+      if (!walker->ok()) {
         // Early exit: avoid executing handlers if DoWalk() returned
-        return;
+        return Status::OK();
       }
       if (!result.ok()) {
-        HandleError(result.status());
-        return;
+        return result.status();
       }
       const auto& outcome = *result;
       if (!outcome.IsSuccess()) {
-        Status st = walker->error_handler_(outcome.GetError());
-        HandleError(std::move(st));
-        return;
+        {
+          auto guard = walker->mutex_.Lock();
+          return walker->error_handler_(outcome.GetError());
+        }
       }
-      HandleResult(outcome.GetResult());
+      return HandleResult(outcome.GetResult());
     }
 
     void SpawnListObjectsV2() {
       auto walker = this->walker;
       auto req = this->req;
-      auto maybe_fut = walker->io_context_.executor()->Submit(
-          walker->io_context_.stop_token(),
-          [walker, req]() { return walker->client_->ListObjectsV2(req); });
-      if (!maybe_fut.ok()) {
-        HandleError(maybe_fut.status());
-        return;
-      }
-      maybe_fut->AddCallback(*this);
+      auto cb = *this;

Review comment:
       If you're capturing `*this` by value then you don't need to capture 
`req` and `walker` as well.




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

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to