lidavidm commented on code in PR #13912:
URL: https://github.com/apache/arrow/pull/13912#discussion_r953686946


##########
cpp/src/arrow/util/async_util.cc:
##########
@@ -19,186 +19,334 @@
 
 #include "arrow/util/future.h"
 #include "arrow/util/logging.h"
+#include "arrow/util/make_unique.h"
+
+#include <deque>
+#include <iostream>
+#include <list>
+#include <mutex>
 
 namespace arrow {
-namespace util {
 
-AsyncDestroyable::AsyncDestroyable() : on_closed_(Future<>::Make()) {}
+using internal::make_unique;
 
-#ifndef NDEBUG
-AsyncDestroyable::~AsyncDestroyable() {
-  DCHECK(constructed_correctly_) << "An instance of AsyncDestroyable must be 
created by "
-                                    "MakeSharedAsync or MakeUniqueAsync";
-}
-#else
-AsyncDestroyable::~AsyncDestroyable() = default;
-#endif
-
-void AsyncDestroyable::Destroy() {
-  DoDestroy().AddCallback([this](const Status& st) {
-    on_closed_.MarkFinished(st);
-    delete this;
-  });
-}
+namespace util {
 
-Status AsyncTaskGroup::AddTask(std::function<Result<Future<>>()> task) {
-  auto guard = mutex_.Lock();
-  if (finished_adding_) {
-    return Status::Cancelled("Ignoring task added after the task group has 
been ended");
-  }
-  if (!err_.ok()) {
-    return err_;
-  }
-  Result<Future<>> maybe_task_fut = task();
-  if (!maybe_task_fut.ok()) {
-    err_ = maybe_task_fut.status();
-    return err_;
-  }
-  return AddTaskUnlocked(*maybe_task_fut, std::move(guard));
-}
+class ThrottleImpl : public AsyncTaskScheduler::Throttle {
+ public:
+  ThrottleImpl(int max_concurrent_cost) : available_cost_(max_concurrent_cost) 
{}
 
-Result<bool> 
AsyncTaskGroup::AddTaskIfNotEnded(std::function<Result<Future<>>()> task) {
-  auto guard = mutex_.Lock();
-  if (finished_adding_) {
-    return false;
-  }
-  if (!err_.ok()) {
-    return err_;
-  }
-  Result<Future<>> maybe_task_fut = task();
-  if (!maybe_task_fut.ok()) {
-    err_ = maybe_task_fut.status();
-    return err_;
+  util::optional<Future<>> TryAcquire(int amt) override {
+    std::lock_guard<std::mutex> lk(mutex_);
+    if (backoff_.is_valid()) {
+      return backoff_;
+    }
+    if (amt <= available_cost_) {
+      available_cost_ -= amt;
+      return nullopt;
+    }
+    backoff_ = Future<>::Make();
+    return backoff_;
   }
-  ARROW_RETURN_NOT_OK(AddTaskUnlocked(*maybe_task_fut, std::move(guard)));
-  return true;
-}
 
-Status AsyncTaskGroup::AddTaskUnlocked(const Future<>& task_fut,
-                                       util::Mutex::Guard guard) {
-  // If the task is already finished there is nothing to track so lets save
-  // some work and return early
-  if (task_fut.is_finished()) {
-    err_ &= task_fut.status();
-    return err_;
+  void Release(int amt) override {
+    Future<> backoff_to_fulfill;
+    {
+      std::lock_guard<std::mutex> lk(mutex_);
+      available_cost_ += amt;
+      if (backoff_.is_valid()) {
+        backoff_to_fulfill = std::move(backoff_);
+      }
+    }
+    if (backoff_to_fulfill.is_valid()) {
+      backoff_to_fulfill.MarkFinished();
+    }
   }
-  running_tasks_++;
-  guard.Unlock();
-  task_fut.AddCallback([this](const Status& st) {
-    auto guard = mutex_.Lock();
-    err_ &= st;
-    if (--running_tasks_ == 0 && finished_adding_) {
-      guard.Unlock();
-      all_tasks_done_.MarkFinished(err_);
-    }
-  });
-  return Status::OK();
+
+ private:
+  std::mutex mutex_;
+  int available_cost_;
+  Future<> backoff_;
+};
+
+std::unique_ptr<AsyncTaskScheduler::Throttle> AsyncTaskScheduler::MakeThrottle(
+    int max_concurrent_cost) {
+  return make_unique<ThrottleImpl>(max_concurrent_cost);
 }
 
-Status AsyncTaskGroup::AddTask(const Future<>& task_fut) {
-  auto guard = mutex_.Lock();
-  if (finished_adding_) {
-    return Status::Cancelled("Ignoring task added after the task group has 
been ended");
-  }
-  if (!err_.ok()) {
-    return err_;
+namespace {
+
+// Very basic FIFO queue
+class FifoQueue : public AsyncTaskScheduler::Queue {
+  using Task = AsyncTaskScheduler::Task;
+  void Push(std::unique_ptr<Task> task) override { 
tasks_.push_back(std::move(task)); }
+
+  std::unique_ptr<Task> Pop() override {
+    std::unique_ptr<Task> task = std::move(tasks_.front());
+    tasks_.pop_front();
+    return task;
   }
-  return AddTaskUnlocked(task_fut, std::move(guard));
-}
 
-Result<bool> AsyncTaskGroup::AddTaskIfNotEnded(const Future<>& task_fut) {
-  auto guard = mutex_.Lock();
-  if (finished_adding_) {
-    return false;
+  const Task& Peek() override { return *tasks_.front(); }
+
+  bool Empty() override { return tasks_.empty(); }
+
+  void Purge() override { tasks_.clear(); }
+
+ private:
+  std::list<std::unique_ptr<Task>> tasks_;

Review Comment:
   Not a big deal, I just prefer to avoid the linked list unless we need the 
pointer stability it provides. 



##########
cpp/src/arrow/util/async_util_benchmark.cc:
##########
@@ -0,0 +1,19 @@
+// 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.
+#include "benchmark/benchmark.h"
+
+#include "arrow/util/async_util.h"

Review Comment:
   Do you mean to fill in the benchmark?



-- 
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: [email protected]

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

Reply via email to