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



##########
File path: cpp/src/arrow/util/async_generator.h
##########
@@ -177,6 +364,126 @@ class TransformingGenerator {
   std::shared_ptr<TransformingGeneratorState> state_;
 };
 
+/// \brief Transforms an async generator using a transformer function 
returning a new
+/// AsyncGenerator
+///
+/// The transform function here behaves exactly the same as the transform 
function in
+/// MakeTransformedIterator and you can safely use the same transform function 
to
+/// transform both synchronous and asynchronous streams.
+///
+/// This generator is not async-reentrant
+///
+/// This generator may queue up to 1 instance of T
+template <typename T, typename V>
+AsyncGenerator<V> MakeAsyncGenerator(AsyncGenerator<T> generator,
+                                     Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+/// \see MakeSerialReadaheadGenerator
+template <typename T>
+class SerialReadaheadGenerator {
+ public:
+  SerialReadaheadGenerator(AsyncGenerator<T> source_generator, int 
max_readahead)
+      : state_(std::make_shared<State>(std::move(source_generator), 
max_readahead)) {}
+
+  Future<T> operator()() {
+    if (state_->first) {
+      // Lazy generator, need to wait for the first ask to prime the pump
+      state_->first = false;
+      auto next = state_->source();
+      return next.Then(Callback{state_});
+    }
+
+    // This generator is not async-reentrant.  We won't be called until the 
last
+    // future finished so we know there is something in the queue
+    auto finished = state_->finished.load();
+    if (finished && state_->readahead_queue.IsEmpty()) {
+      return Future<T>::MakeFinished(IterationTraits<T>::End());
+    }
+
+    auto next_ptr = state_->readahead_queue.FrontPtr();
+    auto next = std::move(**next_ptr);
+    state_->readahead_queue.PopFront();
+
+    auto last_available = state_->spaces_available.fetch_add(1);
+    if (last_available == 0 && !finished) {
+      // Reader idled out, we need to restart it
+      ARROW_RETURN_NOT_OK(state_->Pump(state_));
+    }
+    return next;
+  }
+
+ private:
+  struct State {
+    State(AsyncGenerator<T> source_, int max_readahead)
+        : first(true),
+          source(std::move(source_)),
+          finished(false),
+          spaces_available(max_readahead),
+          readahead_queue(max_readahead) {}
+
+    Status Pump(const std::shared_ptr<State>& self) {
+      // Can't do readahead_queue.write(source().Then(Callback{self})) because 
then the
+      // callback might run immediately and add itself to the queue before 
this gets added
+      // to the queue messing up the order
+      auto next_slot = std::make_shared<Future<T>>();
+      auto written = readahead_queue.Write(next_slot);
+      if (!written) {
+        return Status::UnknownError("Could not write to readahead_queue");
+      }
+      *next_slot = source().Then(Callback{self});
+      return Status::OK();
+    }
+
+    // Only accessed by the consumer end
+    bool first;
+    // Accessed by both threads
+    AsyncGenerator<T> source;
+    std::atomic<bool> finished;
+    std::atomic<uint32_t> spaces_available;
+    util::SpscQueue<std::shared_ptr<Future<T>>> readahead_queue;
+  };
+
+  struct Callback {
+    Result<T> operator()(const Result<T>& maybe_next) {
+      if (!maybe_next.ok()) {
+        state_->finished.store(true);
+        return maybe_next;
+      }
+      const auto& next = *maybe_next;
+      if (IterationTraits<T>::IsEnd(next)) {
+        state_->finished = true;
+        return maybe_next;
+      }
+      auto last_available = state_->spaces_available.fetch_sub(1);
+      if (last_available > 1) {

Review comment:
       Good catch.  It was indeed an off-by-one.  There were two issues, 
`spaces_available` was not accounting for the "in-flight" request and 
ProducerConsumerQueue actually has `size-1` "usable" slots (which is a little 
misleading, size refers to the size in memory and not the functional size).




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to