westonpace commented on a change in pull request #9779:
URL: https://github.com/apache/arrow/pull/9779#discussion_r599852706



##########
File path: cpp/src/arrow/util/async_generator.h
##########
@@ -278,6 +278,147 @@ AsyncGenerator<V> MakeMappedGenerator(AsyncGenerator<T> 
source_generator,
   return MappingGenerator<T, V>(std::move(source_generator), std::move(map));
 }
 
+template <typename T, typename Comp, typename IsNext>
+class SequencingGenerator {
+ public:
+  SequencingGenerator(AsyncGenerator<T> source, Comp compare, IsNext is_next,
+                      T initial_value)
+      : state_(std::make_shared<State>(std::move(source), std::move(compare),
+                                       std::move(is_next), 
std::move(initial_value))) {}
+
+  Future<T> operator()() {
+    util::optional<Result<T>> result;
+    {
+      auto guard = state_->mutex.Lock();
+      if (!state_->queue.empty()) {
+        if (!state_->queue.top().ok() ||
+            state_->is_next(state_->previous_value, *state_->queue.top())) {
+          result = std::move(state_->queue.top());
+          if (result->ok()) {
+            state_->previous_value = **result;
+          }
+          state_->queue.pop();
+        }
+      }
+      if (!result.has_value()) {
+        if (state_->finished) {
+          return AsyncGeneratorEnd<T>();
+        }
+        auto new_waiting_fut = Future<T>::Make();
+        state_->waiting_future = new_waiting_fut;
+        guard.Unlock();
+        state_->source().AddCallback(Callback{state_});
+        return new_waiting_fut;
+      }
+      return Future<T>::MakeFinished(*result);
+    }
+  }
+
+ private:
+  struct WrappedComp {
+    bool operator()(const Result<T>& left, const Result<T>& right) {
+      if (!left.ok() || !right.ok()) {
+        return false;
+      }
+      return compare(*left, *right);
+    }
+    Comp compare;
+  };
+
+  struct State {
+    State(AsyncGenerator<T> source, Comp compare, IsNext is_next, T 
initial_value)
+        : source(std::move(source)),
+          is_next(std::move(is_next)),
+          previous_value(std::move(initial_value)),
+          waiting_future(),
+          queue(WrappedComp{compare}),
+          finished(false),
+          mutex() {}
+
+    AsyncGenerator<T> source;
+    IsNext is_next;
+    T previous_value;
+    Future<T> waiting_future;
+    std::priority_queue<Result<T>, std::vector<Result<T>>, WrappedComp> queue;
+    bool finished;
+    util::Mutex mutex;
+  };
+
+  class Callback {
+   public:
+    explicit Callback(std::shared_ptr<State> state) : state_(std::move(state)) 
{}
+
+    void operator()(const Result<T> result) {
+      Future<T> to_deliver;
+      bool finished;
+      {
+        auto guard = state_->mutex.Lock();
+        bool ready_to_deliver = false;
+        if (!result.ok()) {
+          while (!state_->queue.empty()) {
+            state_->queue.pop();
+          }
+          ready_to_deliver = true;
+          state_->finished = true;
+        } else if (IsIterationEnd<T>(result.ValueUnsafe())) {
+          ready_to_deliver = state_->queue.empty();
+          state_->finished = true;
+        } else {
+          ready_to_deliver = state_->is_next(state_->previous_value, *result);
+        }
+
+        if (ready_to_deliver && state_->waiting_future.is_valid()) {
+          to_deliver = state_->waiting_future;
+          if (result.ok()) {
+            state_->previous_value = *result;
+          }
+        } else {
+          state_->queue.push(result);
+        }
+        finished = state_->finished;
+      }
+      if (to_deliver.is_valid()) {
+        to_deliver.MarkFinished(result);
+      } else {
+        if (!finished) {

Review comment:
       I capture finished while I'm still in the critical section.  I can't 
call `AddCallback` or `MarkFinished` while holding the mutex.  So the first 
block is figuring out which action to take (queue, do nothing, mark existing 
finished) and the second block is performing the action.




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