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 2021/01/04 18:25:29 UTC

[GitHub] [arrow] westonpace opened a new pull request #9095: Feature/arrow 10183 2

westonpace opened a new pull request #9095:
URL: https://github.com/apache/arrow/pull/9095


   


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r572267087



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,382 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;

Review comment:
       I have gone back and forth a little (hence this discrepancy). I don't think asynchronous iterators are terribly common in general.  When they are used (e.g. javascript) it is in conjunction with syntactic "await" sugar to create something that can actually be iterated (e.g. `for await...of`).  The only way to truly "iterate" these chains is to use the Collect/Visit utilities (or the underlying Loop) which I feel isn't quite the same thing.  So asynchronous iterator isn't an ideal name.
   
   Generator is used in python for creating an iterator with yield statements which is an entirely separate concept.  It's used exactly the same in javascript.  In fact, in javascript there is an "asynchronous generator" which allows you to create asynchronous functions that yield (which is not what we are doing there except perhaps in the transform functions).  So the asynchronous generator name isn't perfect either.
   
   In the FRP world these might be called events and event streams but I think that fits a push-based model better.
   
   I think, in the interest of "perfect is the enemy of good" I will stick with AsyncGenerator everywhere and rename `async_iterator` to `async_generator`.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r570536386



##########
File path: cpp/src/arrow/csv/options.h
##########
@@ -119,6 +119,13 @@ struct ARROW_EXPORT ReadOptions {
 
   /// Whether to use the global CPU thread pool
   bool use_threads = true;
+  /// TODO: This will come out before this gets merged in.  There is not much point in

Review comment:
       Done.  I went ahead and just removed the threaded table reader entirely.  Now that I've done more benchmarking I don't see any reason for keeping it in.




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r566303697



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      util::optional<Result<V>> next = Pump();
+      if (next.has_value()) {
+        return *next;

Review comment:
       But `*next` is not a rvalue. If you were returning `next`, presumably the move wouldn't be required.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564736538



##########
File path: cpp/src/arrow/csv/reader_benchmark.cc
##########
@@ -0,0 +1,139 @@
+// 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 <string>
+
+#include "arrow/buffer.h"
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+class SlowInputStream : public io::InputStream {
+ public:
+  explicit SlowInputStream(std::shared_ptr<io::BufferReader> target, int64_t latency_ms)
+      : target_(std::move(target)) {
+    latency_s_ = static_cast<double>(latency_ms) / 1000.0;
+  }
+  virtual ~SlowInputStream() {}
+
+  Result<util::string_view> Peek(int64_t nbytes) override {
+    return target_->Peek(nbytes);
+  }
+  bool supports_zero_copy() const override { return target_->supports_zero_copy(); }
+  Status Close() override { return target_->Close(); }
+  Status Abort() override { return target_->Abort(); }
+  Result<int64_t> Tell() const override { return target_->Tell(); }
+  bool closed() const override { return target_->closed(); }
+  Result<int64_t> Read(int64_t nbytes, void* out) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes, out);
+  }
+  Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes);
+  }
+  Status Seek(int64_t pos) { return target_->Seek(pos); }
+
+ private:
+  std::shared_ptr<io::BufferReader> target_;
+  double latency_s_;
+};
+
+static ReadOptions CreateReadOptions(bool use_threads, bool use_async) {
+  auto result = csv::ReadOptions::Defaults();
+  result.use_threads = use_threads;
+  result.legacy_blocking_reads = !use_async;
+  // Simulate larger files by using smaller block files so the impact of multiple
+  // blocks is seen but we don't have to spend the time waiting on the large I/O
+  result.block_size = (1 << 20) / 100;
+  return result;
+}
+
+static std::shared_ptr<SlowInputStream> CreateStreamReader(std::shared_ptr<Buffer> buffer,
+                                                           int64_t latency_ms) {
+  auto buffer_reader = std::make_shared<io::BufferReader>(buffer);
+  return std::make_shared<SlowInputStream>(buffer_reader, latency_ms);
+}
+
+static void BenchmarkReader(benchmark::State& state, bool use_threads, bool use_async) {
+  auto latency_ms = state.range(0);
+  auto num_rows = state.range(1);
+  auto num_files = state.range(2);
+  if (num_files > 5 && use_threads && !use_async) {
+    state.SkipWithError("Would deadlock");
+  }
+  auto input_buffer = *MakeSampleCsvBuffer(num_rows);
+  // Hard coding # of threads so we don't deadlock if there are too few cores

Review comment:
       I don't think we want to include the benchmark at the end of the day.  I created it originally to be able to demonstrate the nested deadlock case encountered by the dataset API.  I wrote it for my own benefit to both confirm that the async table reader was avoiding the deadlock and show that it can outperform the threaded table reader since threads aren't wasted blocking on I/O.  So it is mimicking the dataset API in two ways that don't make sense for a benchmark and, somewhat intentionally, cause the threaded table reader to underperform.
   
   First, the table reader is running inside a thread pool thread for the threaded/serial case.  This simulates the way the dataset API currently burns a thread waiting for the threaded table reader in much the same way.
   
   Second, we are fixing the thread pool to size 6.
   
   Eliminating that "waiter thread task" would prevent the deadlock entirely.  With that in there however, as soon as you have as many waiters (one per file) as you have threads in your pool then you will deadlock.
   
   Thus, this benchmark is probably more of a temporary demo than it is a benchmark.  In the future, once the dataset logic has moved over to async, we can put in a real benchmark showing the actual gain.  I will rearrange the commits so that this demo/benchmark is a separate branch built on top of the 10183 branch that can be optionally checked out.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565956846



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -556,6 +559,38 @@ inline bool WaitForAll(const std::vector<Future<T>*>& futures,
   return waiter->Wait(seconds);
 }
 
+/// \brief Create a Future which completes when all of `futures` complete.
+///
+/// The future's result is a vector of the results of `futures`.
+/// Note that this future will never be marked "failed"; failed results
+/// will be stored in the result vector alongside successful results.
+template <typename T>
+Future<std::vector<Result<T>>> All(std::vector<Future<T>> futures) {
+  struct State {
+    explicit State(std::vector<Future<T>> f)
+        : futures(std::move(f)), n_remaining(futures.size()) {}
+
+    std::vector<Future<T>> futures;
+    std::atomic<size_t> n_remaining;
+  };

Review comment:
       Ok.  Yes, it could be done with the waiters if the waiter were given a "finished future" and it marked it complete when the futures all finished.  Then a WaitForAllAsync could be added.  I'm not sure there would be any advantage though.  The waiter is basically a named callback from the future's perspective.  I'm happy to proceed with this if you think it'll be cleaner.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565968871



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +615,74 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>
+Future<BreakValueType> Loop(Iterate iterate) {
+  auto break_fut = Future<BreakValueType>::Make();
+
+  struct Callback {
+    bool CheckForTermination(const Result<Control>& maybe_control) {
+      if (!maybe_control.ok() || maybe_control->IsBreak()) {
+        Result<BreakValueType> maybe_break = maybe_control.Map(Control::MoveBreakValue);
+        break_fut.MarkFinished(std::move(maybe_break));
+        return true;
+      }
+      return false;
+    }
+
+    void operator()(const Result<Control>& maybe_control) && {
+      if (CheckForTermination(maybe_control)) return;
+
+      auto control_fut = iterate();
+      while (control_fut.is_finished()) {
+        // There's no need to AddCallback on a finished future; we can CheckForTermination
+        // now. This also avoids recursion and potential stack overflow.

Review comment:
       I created `TryAddCallback` and created a test case to confirm that using `TryAddCallback` one can completely avoid a synchronous callback execution. 




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



[GitHub] [arrow] github-actions[bot] commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-754142555


   https://issues.apache.org/jira/browse/ARROW-10183


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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565981888



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);

Review comment:
       Ok, thank you.




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



[GitHub] [arrow] bkietz closed pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
bkietz closed pull request #9095:
URL: https://github.com/apache/arrow/pull/9095


   


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565968057



##########
File path: cpp/src/arrow/util/task_group.cc
##########
@@ -135,6 +149,18 @@ class ThreadedTaskGroup : public TaskGroup {
       // before cv.notify_one() has returned
       std::unique_lock<std::mutex> lock(mutex_);
       cv_.notify_one();
+      if (completion_future_.has_value()) {
+        // MarkFinished could be slow.  We don't want to call it while we are holding
+        // the lock.
+        // TODO: If optional is thread safe then we can skip this locking entirely
+        auto future = *completion_future_;
+        auto finished = completion_future_->is_finished();
+        auto status = status_;

Review comment:
       Done.  It is now `const auto&`




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r570536386



##########
File path: cpp/src/arrow/csv/options.h
##########
@@ -119,6 +119,13 @@ struct ARROW_EXPORT ReadOptions {
 
   /// Whether to use the global CPU thread pool
   bool use_threads = true;
+  /// TODO: This will come out before this gets merged in.  There is not much point in

Review comment:
       Done.  I went ahead and just removed the threaded table reader entirely.  Now that I've done more benchmarking I don't see any reason for keeping it in.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565932542



##########
File path: cpp/src/arrow/csv/reader_benchmark.cc
##########
@@ -0,0 +1,139 @@
+// 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 <string>
+
+#include "arrow/buffer.h"
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+class SlowInputStream : public io::InputStream {
+ public:
+  explicit SlowInputStream(std::shared_ptr<io::BufferReader> target, int64_t latency_ms)
+      : target_(std::move(target)) {
+    latency_s_ = static_cast<double>(latency_ms) / 1000.0;
+  }
+  virtual ~SlowInputStream() {}
+
+  Result<util::string_view> Peek(int64_t nbytes) override {
+    return target_->Peek(nbytes);
+  }
+  bool supports_zero_copy() const override { return target_->supports_zero_copy(); }
+  Status Close() override { return target_->Close(); }
+  Status Abort() override { return target_->Abort(); }
+  Result<int64_t> Tell() const override { return target_->Tell(); }
+  bool closed() const override { return target_->closed(); }
+  Result<int64_t> Read(int64_t nbytes, void* out) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes, out);
+  }
+  Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes);
+  }
+  Status Seek(int64_t pos) { return target_->Seek(pos); }
+
+ private:
+  std::shared_ptr<io::BufferReader> target_;
+  double latency_s_;
+};
+
+static ReadOptions CreateReadOptions(bool use_threads, bool use_async) {
+  auto result = csv::ReadOptions::Defaults();
+  result.use_threads = use_threads;
+  result.legacy_blocking_reads = !use_async;
+  // Simulate larger files by using smaller block files so the impact of multiple
+  // blocks is seen but we don't have to spend the time waiting on the large I/O
+  result.block_size = (1 << 20) / 100;
+  return result;
+}
+
+static std::shared_ptr<SlowInputStream> CreateStreamReader(std::shared_ptr<Buffer> buffer,
+                                                           int64_t latency_ms) {
+  auto buffer_reader = std::make_shared<io::BufferReader>(buffer);
+  return std::make_shared<SlowInputStream>(buffer_reader, latency_ms);
+}
+
+static void BenchmarkReader(benchmark::State& state, bool use_threads, bool use_async) {
+  auto latency_ms = state.range(0);
+  auto num_rows = state.range(1);
+  auto num_files = state.range(2);
+  if (num_files > 5 && use_threads && !use_async) {

Review comment:
       Answered above.




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



[GitHub] [arrow] ursabot commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-777085236


   ubuntu-20.04-x86_64: https://conbench.ursa.dev/compare/runs/3aa6839d-e95c-4364-8e91-6edad18f0a72...8d78aff4-4c59-4a0a-98b2-75ca5f439243/
   dgx-ubuntu-18.04-x86_64: https://conbench.ursa.dev/compare/runs/cde6a063-c0f9-4364-8295-8742f5e44649...e91b50f5-bdad-4cf8-86e7-8ccc25016e74/


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565932196



##########
File path: cpp/src/arrow/csv/reader_benchmark.cc
##########
@@ -0,0 +1,139 @@
+// 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 <string>
+
+#include "arrow/buffer.h"
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+class SlowInputStream : public io::InputStream {
+ public:
+  explicit SlowInputStream(std::shared_ptr<io::BufferReader> target, int64_t latency_ms)
+      : target_(std::move(target)) {
+    latency_s_ = static_cast<double>(latency_ms) / 1000.0;
+  }
+  virtual ~SlowInputStream() {}
+
+  Result<util::string_view> Peek(int64_t nbytes) override {
+    return target_->Peek(nbytes);
+  }
+  bool supports_zero_copy() const override { return target_->supports_zero_copy(); }
+  Status Close() override { return target_->Close(); }
+  Status Abort() override { return target_->Abort(); }
+  Result<int64_t> Tell() const override { return target_->Tell(); }
+  bool closed() const override { return target_->closed(); }
+  Result<int64_t> Read(int64_t nbytes, void* out) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes, out);
+  }
+  Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes);
+  }
+  Status Seek(int64_t pos) { return target_->Seek(pos); }
+
+ private:
+  std::shared_ptr<io::BufferReader> target_;
+  double latency_s_;
+};
+
+static ReadOptions CreateReadOptions(bool use_threads, bool use_async) {
+  auto result = csv::ReadOptions::Defaults();
+  result.use_threads = use_threads;
+  result.legacy_blocking_reads = !use_async;
+  // Simulate larger files by using smaller block files so the impact of multiple
+  // blocks is seen but we don't have to spend the time waiting on the large I/O

Review comment:
       I've removed this benchmark from the PR.  Those fixed costs is what I was hoping to emphasize since I was looking for to see how much overhead the futures implementation was adding and wanted to minimize the reading/parsing costs (since this PR didn't change those functions at all).




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565970329



##########
File path: cpp/src/arrow/util/thread_pool.h
##########
@@ -87,6 +87,21 @@ class ARROW_EXPORT Executor {
     return SpawnReal(hints, std::forward<Function>(func));
   }
 
+  template <typename T>
+  Future<T> Transfer(Future<T> future) {
+    auto transferred = Future<T>::Make();
+    future.AddCallback([this, transferred](const Result<T>& result) mutable {
+      Result<T> result_copy(result);
+      auto spawn_status = Spawn([transferred, result_copy]() mutable {

Review comment:
       Just to elaborate.  A future can have multiple callbacks so it provides a reference to the result.  No callback is allowed to steal the result.  I think I previously discussed we could introduce the concept of a "final callback" which gets the moved result and not a reference.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565937198



##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NTASKS = 100;
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+
+  std::vector<Future<std::shared_ptr<Table>>> task_futures(NTASKS);
+  for (int i = 0; i < NTASKS; i++) {
+    auto input = std::make_shared<io::BufferReader>(table_buffer);
+    ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+    task_futures[i] = reader->ReadAsync();
+  }
+  auto combined_future = All(task_futures);
+  combined_future.Wait();
+
+  ASSERT_OK_AND_ASSIGN(std::vector<Result<std::shared_ptr<Table>>> results,
+                       combined_future.result());
+  for (auto&& result : results) {
+    ASSERT_OK_AND_ASSIGN(auto table, result);
+    ASSERT_EQ(NROWS, table->num_rows());
+  }
+}
+
+void TestNestedParallelism(
+    std::shared_ptr<internal::ThreadPool> thread_pool,
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+  auto input = std::make_shared<io::BufferReader>(table_buffer);
+  ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+
+  Future<std::shared_ptr<Table>> table_future;
+
+  auto read_task = [&reader, &table_future]() mutable {
+    table_future = reader->ReadAsync();
+    return Status::OK();
+  };
+  ASSERT_OK_AND_ASSIGN(auto future, thread_pool->Submit(read_task));
+  ASSERT_TRUE(future.Wait(1));
+
+  if (future.is_finished()) {

Review comment:
       I have added ASSERT_FINISHES_OK (future must finish within 2 seconds with an OK status) and ASSERT_FINISHES_OK_AND_ASSIGN (future must finish within 2 seconds with an OK status and assigns the result).  There should no longer be any waits followed by an immediate "if finished"




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564828654



##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,9 +832,203 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(3, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(Status::IOError("XYZ"), results[1].status());
+    ASSERT_EQ(3, *results[2]);
+  }));
+
+  f1.MarkFinished(1);
+  f2.MarkFinished(Status::IOError("XYZ"));
+  f3.MarkFinished(3);
+
+  AssertFinished(combined);
+}
+
+TEST(FutureLoopTest, Sync) {
+  struct {
+    int i = 0;
+    Future<int> Get() { return Future<int>::MakeFinished(i++); }
+  } IntSource;
+
+  bool do_fail = false;
+  std::vector<int> ints;
+  auto loop_body = [&] {
+    return IntSource.Get().Then([&](int i) -> Result<ControlFlow<int>> {
+      if (do_fail && i == 3) {
+        return Status::IOError("xxx");
+      }
+
+      if (i == 5) {
+        int sum = 0;
+        for (int i : ints) sum += i;
+        return Break(sum);
+      }
+
+      ints.push_back(i);
+      return Continue();
+    });
+  };
+
+  {
+    auto sum_fut = Loop(loop_body);
+    AssertSuccessful(sum_fut);
+
+    ASSERT_OK_AND_ASSIGN(auto sum, sum_fut.result());
+    ASSERT_EQ(sum, 0 + 1 + 2 + 3 + 4);
+  }
+
+  {
+    do_fail = true;
+    IntSource.i = 0;
+    auto sum_fut = Loop(loop_body);
+    AssertFailed(sum_fut);
+    ASSERT_RAISES(IOError, sum_fut.result());
+  }
+}
+
+TEST(FutureLoopTest, EmptyBreakValue) {
+  Future<> none_fut =
+      Loop([&] { return Future<>::MakeFinished().Then([&](...) { return Break(); }); });
+  AssertSuccessful(none_fut);
+}
+
+TEST(FutureLoopTest, MoveOnlyBreakValue) {
+  Future<MoveOnlyDataType> one_fut = Loop([&] {
+    return Future<int>::MakeFinished(1).Then(
+        [&](int i) { return Break(MoveOnlyDataType(i)); });
+  });
+  AssertSuccessful(one_fut);
+  ASSERT_OK_AND_ASSIGN(auto one, std::move(one_fut).result());
+  ASSERT_EQ(one, 1);
+}
+
+TEST(FutureLoopTest, StackOverflow) {
+  // Looping over futures is normally a rather recursive task.  If the futures complete
+  // synchronously (because they are already finished) it could lead to a stack overflow
+  // if care is not taken.
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 1000000) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());
+    }
+    return Future<ControlFlow<int>>::MakeFinished(Break(-1));
+  };
+  auto loop_fut = Loop(loop_body);
+  ASSERT_TRUE(loop_fut.Wait(0.1));
+}
+
+TEST(FutureLoopTest, AllowsBreakFutToBeDiscarded) {
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 10) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());
+    }
+    return Future<ControlFlow<int>>::MakeFinished(Break(-1));
+  };
+  auto loop_fut = Loop(loop_body).Then([](...) { return Status::OK(); });
+  ASSERT_TRUE(loop_fut.Wait(0.1));
+}
+
+TEST(FutureLoopTest, EmptyLoop) {
+  auto loop_body = []() -> Future<ControlFlow<int>> {
+    return Future<ControlFlow<int>>::MakeFinished(Break(0));
+  };
+  auto loop_fut = Loop(loop_body);
+  ASSERT_TRUE(loop_fut.Wait(0.1));
+  if (loop_fut.is_finished()) {

Review comment:
       `ASSERT_TRUE` should definitely return from the function. It's the difference with `EXPECT_TRUE`.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564788723



##########
File path: cpp/src/arrow/result.h
##########
@@ -331,6 +332,7 @@ class ARROW_MUST_USE_TYPE Result : public util::EqualityComparable<Result<T>> {
     return ValueUnsafe();
   }
   T& operator*() & { return ValueOrDie(); }
+  T* operator->() { return &ValueOrDie(); }

Review comment:
       @bkietz I'm pretty sure this is your contribution.  Is this something I should add?  I'm not entirely sure of the semantics.  I think you mean...
   
   ```
   T operator*() && { return MoveValueOrDie(); }
   ```
   
   ...I think its use would be limited.  Given that ARROW_ASSIGN_OR_RAISE will avoid having to use the * operator most of the time.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r566302012



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      util::optional<Result<V>> next = Pump();
+      if (next.has_value()) {
+        return *next;

Review comment:
       RVO is optional but I believe the the implicit move is not.  If the returned type is not a reference then it will first try and return it as an rvalue and only if that fails will it consider copy.  So any std::move will be redundant at best and harmful (preventing RVO) at worst.
   
   Otherwise we would have to determine ourselves when RVO applies and when it does not so that we know whether to apply std::move or not.
   
   I guess, what I am asking is...what makes this return different than the dozens of other returns where I do not move the return value?  Why is it needed here and not elsewhere?  What are the rules I use to decide when to move a return value?




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564745648



##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -837,20 +885,121 @@ class ThreadedTableReader : public BaseTableReader {
   }
 
  protected:
-  ThreadPool* thread_pool_;
+  Executor* thread_pool_;
+  Iterator<std::shared_ptr<Buffer>> buffer_iterator_;
+};
+
+class AsyncThreadedTableReader
+    : public BaseTableReader,
+      public std::enable_shared_from_this<AsyncThreadedTableReader> {
+ public:
+  using BaseTableReader::BaseTableReader;
+
+  AsyncThreadedTableReader(MemoryPool* pool, std::shared_ptr<io::InputStream> input,
+                           const ReadOptions& read_options,
+                           const ParseOptions& parse_options,
+                           const ConvertOptions& convert_options, Executor* thread_pool)
+      : BaseTableReader(pool, input, read_options, parse_options, convert_options),
+        thread_pool_(thread_pool) {}
+
+  ~AsyncThreadedTableReader() override {
+    if (task_group_) {
+      // In case of error, make sure all pending tasks are finished before
+      // we start destroying BaseTableReader members
+      ARROW_UNUSED(task_group_->Finish());
+    }
+  }
+
+  Status Init() override {
+    ARROW_ASSIGN_OR_RAISE(auto istream_it,
+                          io::MakeInputStreamIterator(input_, read_options_.block_size));
+
+    ARROW_ASSIGN_OR_RAISE(auto bg_it,
+                          MakeBackgroundIterator(std::move(istream_it), thread_pool_));
+
+    int32_t block_queue_size = thread_pool_->GetCapacity();
+    auto rh_it = AddReadahead(bg_it, block_queue_size);
+    buffer_generator_ = CSVBufferIterator::MakeAsync(std::move(rh_it));
+    return Status::OK();
+  }
+
+  Result<std::shared_ptr<Table>> Read() override { return ReadAsync().result(); }
+
+  Future<std::shared_ptr<Table>> ReadAsync() override {
+    task_group_ = internal::TaskGroup::MakeThreaded(thread_pool_);
+
+    auto self = shared_from_this();
+    return ProcessFirstBuffer().Then([self](const std::shared_ptr<Buffer> first_buffer) {
+      auto block_generator = ThreadedBlockReader::MakeAsyncIterator(
+          self->buffer_generator_, MakeChunker(self->parse_options_),
+          std::move(first_buffer));
+
+      std::function<Status(util::optional<CSVBlock>)> block_visitor =
+          [self](util::optional<CSVBlock> maybe_block) -> Status {
+        DCHECK(!maybe_block->consume_bytes);

Review comment:
       See previous comment.  The Visit function is never passed the end token, there is no need.  However, since the user is allowed to specify their own end token, this use of util::optional is unavoidable.  For example, a user could create an iterator of TestInt where the end token is simply the TestInt with value -999.  Now there is no "wrapping" going on (in the way that  `util::optional` wraps `CSVBlock`) so there is nothing to unwrap when calling the `Visit` function.
   
   On the other hand, if we assume `util::optional` is always used for iterators of non-pointer types we could redefine visit to take in T instead.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r555261490



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -129,11 +130,47 @@ template <typename T>
 inline Iterator<T> EmptyIt() {
   return MakeEmptyIterator<T>();
 }
-
 inline Iterator<TestInt> VectorIt(std::vector<TestInt> v) {
   return MakeVectorIterator<TestInt>(std::move(v));
 }
 
+std::function<Future<TestInt>()> AsyncVectorIt(std::vector<TestInt> v) {
+  auto index = std::make_shared<size_t>(0);
+  auto vec = std::make_shared<std::vector<TestInt>>(std::move(v));
+  return [index, vec]() -> Future<TestInt> {
+    if (*index >= vec->size()) {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+    auto next = (*vec)[*index];
+    (*index)++;
+    return Future<TestInt>::MakeFinished(next);
+  };

Review comment:
       This is good.  I changed all of the tests in iterator_test.cc to make use of captured lambda state instead of shared pointer state.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r566278194



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.
+template <typename T>
+class BackgroundIterator {
+  using PromiseType = typename detail::BackgroundIteratorPromise<T>;
+
+ public:
+  explicit BackgroundIterator(Iterator<T> it, internal::Executor* executor)
+      : it_(new Iterator<T>(std::move(it))),
+        queue_(new detail::ReadaheadQueue(0)),
+        executor_(executor),
+        done_() {}
+
+  ~BackgroundIterator() {
+    if (queue_) {
+      // Make sure the queue doesn't call any promises after this object
+      // is destroyed.
+      queue_->EnsureShutdownOrDie();
+    }
+  }
+
+  ARROW_DEFAULT_MOVE_AND_ASSIGN(BackgroundIterator);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(BackgroundIterator);
+
+  Future<T> operator()() {
+    if (done_) {
+      return Future<T>::MakeFinished(IterationTraits<T>::End());
+    }
+    auto promise = std::unique_ptr<PromiseType>(new PromiseType{it_.get()});
+    auto result = Future<T>(promise->out_);
+    // TODO: Need a futuristic version of ARROW_RETURN_NOT_OK
+    auto append_status = queue_->Append(
+        static_cast<std::unique_ptr<detail::ReadaheadPromise>>(std::move(promise)));
+    if (!append_status.ok()) {
+      return Future<T>::MakeFinished(append_status);
+    }
+
+    result.AddCallback([this](const Result<T>& result) {
+      if (!result.ok() || result.ValueUnsafe() == IterationTraits<T>::End()) {
+        done_ = true;
+      }
+    });
+
+    return executor_->Transfer(result);

Review comment:
       I think that is a good idea.  Then we could have a simple mandate: all functions that return a future should return a future that continues on the caller's executor.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565958445



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +615,74 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>
+Future<BreakValueType> Loop(Iterate iterate) {
+  auto break_fut = Future<BreakValueType>::Make();
+
+  struct Callback {
+    bool CheckForTermination(const Result<Control>& maybe_control) {
+      if (!maybe_control.ok() || maybe_control->IsBreak()) {
+        Result<BreakValueType> maybe_break = maybe_control.Map(Control::MoveBreakValue);
+        break_fut.MarkFinished(std::move(maybe_break));
+        return true;
+      }
+      return false;
+    }
+
+    void operator()(const Result<Control>& maybe_control) && {
+      if (CheckForTermination(maybe_control)) return;
+
+      auto control_fut = iterate();
+      while (control_fut.is_finished()) {
+        // There's no need to AddCallback on a finished future; we can CheckForTermination
+        // now. This also avoids recursion and potential stack overflow.

Review comment:
       I struggled to create a test case that could reproduce this.  It seems like it should be reproducible but even adding a slow callback (so MarkFinished took a long time) it was difficult to get a task to be added and then finish just when I wanted it to.  I have added some logic however so that this race condition should not be possible any longer.




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565171408



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {

Review comment:
       Hmm, I don't think it hurts to minimize copies in any case. Not sure what @bkietz thinks about this?
   
   Also, perhaps the "wrap a callable in a shared_ptr" pattern can be eased with a dedicated primitive?
   




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r555194510



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -35,6 +36,13 @@
 
 namespace arrow {
 
+namespace detail {
+
+template <typename Signature>
+using result_of_t = typename std::result_of<Signature>::type;

Review comment:
       It was unused in iterator.h so I just removed it.  I think I pulled it in because I was using Loop as a base for the transforming iterator but it mutated enough to longer require this.




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



[GitHub] [arrow] westonpace commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-771123067


   @pitrou I believe this is ready for another review.  In addition to fixing your comments:
   
   * Removed rvalue version of Future::result() since this is unsafe (future does not own its result exclusively)
   * Moved the "Transfer" out of the "BackgroundGenerator" so it is explicit as per Zulip discussion
   * Fixed a bug in the transforming generator. It had a this capture that was not valid because the returned async generator pipeline could be moved around and callbacks might still be referencing this.
   * The TryAddCallback test would sometimes fail because it is not valid to do std::function<void(const Result<detail::Empty>&)> callback = [&callback_adder_thread_id](...) (the ... can't be assigned to a std::function in that way)
   


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



[GitHub] [arrow] westonpace edited a comment on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace edited a comment on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-771123067


   @pitrou I believe this is ready for another review.  In addition to fixing your comments:
   
   * Removed rvalue version of Future::result() since this is unsafe (future does not own its result exclusively)
   * Moved the "Transfer" out of the "BackgroundGenerator" so it is explicit as per Zulip discussion
   * Fixed a bug in the transforming generator. It had a `this` capture that was not valid because the returned async generator pipeline could be moved around and callbacks might still be referencing `this`.
   * The TryAddCallback test would sometimes fail because it is not valid to do std::function<void(const Result<detail::Empty>&)> callback = [&callback_adder_thread_id](...) (the ... can't be assigned to a std::function in that way)
   


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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565978816



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {

Review comment:
       Well, at least the templated callable can be inlined, while the `std::function` presumably never will (or at least is much less likely to be inlined).




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r572257746



##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -837,20 +885,121 @@ class ThreadedTableReader : public BaseTableReader {
   }
 
  protected:
-  ThreadPool* thread_pool_;
+  Executor* thread_pool_;
+  Iterator<std::shared_ptr<Buffer>> buffer_iterator_;
+};
+
+class AsyncThreadedTableReader
+    : public BaseTableReader,
+      public std::enable_shared_from_this<AsyncThreadedTableReader> {
+ public:
+  using BaseTableReader::BaseTableReader;
+
+  AsyncThreadedTableReader(MemoryPool* pool, std::shared_ptr<io::InputStream> input,
+                           const ReadOptions& read_options,
+                           const ParseOptions& parse_options,
+                           const ConvertOptions& convert_options, Executor* thread_pool)
+      : BaseTableReader(pool, input, read_options, parse_options, convert_options),
+        thread_pool_(thread_pool) {}
+
+  ~AsyncThreadedTableReader() override {
+    if (task_group_) {
+      // In case of error, make sure all pending tasks are finished before
+      // we start destroying BaseTableReader members
+      ARROW_UNUSED(task_group_->Finish());
+    }
+  }
+
+  Status Init() override {
+    ARROW_ASSIGN_OR_RAISE(auto istream_it,
+                          io::MakeInputStreamIterator(input_, read_options_.block_size));
+
+    ARROW_ASSIGN_OR_RAISE(auto bg_it,
+                          MakeBackgroundIterator(std::move(istream_it), thread_pool_));
+
+    int32_t block_queue_size = thread_pool_->GetCapacity();
+    auto rh_it = AddReadahead(bg_it, block_queue_size);
+    buffer_generator_ = CSVBufferIterator::MakeAsync(std::move(rh_it));
+    return Status::OK();
+  }
+
+  Result<std::shared_ptr<Table>> Read() override { return ReadAsync().result(); }
+
+  Future<std::shared_ptr<Table>> ReadAsync() override {
+    task_group_ = internal::TaskGroup::MakeThreaded(thread_pool_);

Review comment:
       Right.  I had completely misunderstood the purpose of `io::AsyncContext`.  The CPU thread pool is the correct thread pool.
   
   However, we could still use this I/O pool in a different spot.  It could be used instead of a dedicated "readahead" thread.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r555194884



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}

Review comment:
       Done, this is much simpler.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565980103



##########
File path: cpp/src/arrow/util/task_group_test.cc
##########
@@ -244,6 +245,65 @@ void TestNoCopyTask(std::shared_ptr<TaskGroup> task_group) {
   ASSERT_EQ(0, *counter);
 }
 
+void TestFinishNotSticky(std::function<std::shared_ptr<TaskGroup>()> factory) {
+  // If a task is added that runs very quickly it might decrement the task counter back
+  // down to 0 and mark the completion future as complete before all tasks are added.
+  // The "finished future" of the task group could get stuck to complete.
+  const int NTASKS = 100;
+  for (int i = 0; i < NTASKS; ++i) {
+    auto task_group = factory();
+    // Add a task and let it complete
+    task_group->Append([] { return Status::OK(); });
+    // Wait a little bit, if the task group was going to lock the finish hopefully it
+    // would do so here while we wait
+    SleepFor(1e-2);
+
+    // Add a new task that will still be running
+    std::atomic<bool> ready(false);
+    std::mutex m;
+    std::condition_variable cv;
+    task_group->Append([&m, &cv, &ready] {
+      std::unique_lock<std::mutex> lk(m);
+      cv.wait(lk, [&ready] { return ready.load(); });
+      return Status::OK();
+    });
+
+    // Ensure task group not finished already
+    auto finished = task_group->FinishAsync();
+    ASSERT_FALSE(finished.is_finished());
+
+    std::unique_lock<std::mutex> lk(m);
+    ready = true;
+    lk.unlock();
+    cv.notify_one();
+
+    ASSERT_TRUE(finished.Wait(1));

Review comment:
       I added this check.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564765946



##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NTASKS = 100;
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+
+  std::vector<Future<std::shared_ptr<Table>>> task_futures(NTASKS);
+  for (int i = 0; i < NTASKS; i++) {
+    auto input = std::make_shared<io::BufferReader>(table_buffer);
+    ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+    task_futures[i] = reader->ReadAsync();
+  }
+  auto combined_future = All(task_futures);
+  combined_future.Wait();
+
+  ASSERT_OK_AND_ASSIGN(std::vector<Result<std::shared_ptr<Table>>> results,
+                       combined_future.result());
+  for (auto&& result : results) {
+    ASSERT_OK_AND_ASSIGN(auto table, result);
+    ASSERT_EQ(NROWS, table->num_rows());
+  }
+}
+
+void TestNestedParallelism(
+    std::shared_ptr<internal::ThreadPool> thread_pool,
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+  auto input = std::make_shared<io::BufferReader>(table_buffer);
+  ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+
+  Future<std::shared_ptr<Table>> table_future;
+
+  auto read_task = [&reader, &table_future]() mutable {
+    table_future = reader->ReadAsync();
+    return Status::OK();
+  };
+  ASSERT_OK_AND_ASSIGN(auto future, thread_pool->Submit(read_task));
+  ASSERT_TRUE(future.Wait(1));
+
+  if (future.is_finished()) {

Review comment:
       Ideally.  There were a number of places I had to write...
   
   ```
   ASSERT_TRUE(future.Wait(1));
   if (future.is_finished()) {
   // ...
   }
   ```
   
   ...There is confusion between googletest's notion of "assert" (fail test but continue) and the assert in, for example, ASSERT_OK_AND_ASSIGN (abort).  I will create an `ASSERT_FINISHES_AND_ASSIGN` macro that aborts if the future does not finish promptly.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565531144



##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NTASKS = 100;
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+
+  std::vector<Future<std::shared_ptr<Table>>> task_futures(NTASKS);
+  for (int i = 0; i < NTASKS; i++) {
+    auto input = std::make_shared<io::BufferReader>(table_buffer);
+    ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+    task_futures[i] = reader->ReadAsync();
+  }
+  auto combined_future = All(task_futures);
+  combined_future.Wait();
+
+  ASSERT_OK_AND_ASSIGN(std::vector<Result<std::shared_ptr<Table>>> results,
+                       combined_future.result());
+  for (auto&& result : results) {
+    ASSERT_OK_AND_ASSIGN(auto table, result);
+    ASSERT_EQ(NROWS, table->num_rows());
+  }
+}
+
+void TestNestedParallelism(
+    std::shared_ptr<internal::ThreadPool> thread_pool,
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+  auto input = std::make_shared<io::BufferReader>(table_buffer);
+  ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+
+  Future<std::shared_ptr<Table>> table_future;
+
+  auto read_task = [&reader, &table_future]() mutable {
+    table_future = reader->ReadAsync();
+    return Status::OK();
+  };
+  ASSERT_OK_AND_ASSIGN(auto future, thread_pool->Submit(read_task));
+  ASSERT_TRUE(future.Wait(1));
+
+  if (future.is_finished()) {
+    ASSERT_TRUE(table_future.Wait(1));
+    if (table_future.is_finished()) {
+      ASSERT_OK_AND_ASSIGN(auto table, table_future.result());
+      ASSERT_EQ(table->num_rows(), NROWS);
+    }
+  }
+}  // namespace csv
+
+TEST(SerialReaderTests, Stress) {
+  auto task_factory = [](std::shared_ptr<io::InputStream> input_stream) {
+    return TableReader::Make(default_memory_pool(), io::AsyncContext(), input_stream,
+                             ReadOptions::Defaults(), ParseOptions::Defaults(),
+                             ConvertOptions::Defaults());
+  };
+  StressTableReader(task_factory);
+}
+
+TEST(SerialReaderTests, NestedParallelism) {
+  ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1));
+  auto task_factory = [](std::shared_ptr<io::InputStream> input_stream) {
+    return TableReader::Make(default_memory_pool(), io::AsyncContext(), input_stream,
+                             ReadOptions::Defaults(), ParseOptions::Defaults(),
+                             ConvertOptions::Defaults());
+  };
+  TestNestedParallelism(thread_pool, task_factory);
+}
+
+TEST(ThreadedReaderTests, Stress) {
+  ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1));
+  auto task_factory = [&thread_pool](std::shared_ptr<io::InputStream> input_stream)
+      -> Result<std::shared_ptr<TableReader>> {
+    ReadOptions read_options = ReadOptions::Defaults();
+    read_options.use_threads = true;
+    read_options.legacy_blocking_reads = true;
+    auto table_reader = TableReader::Make(
+        default_memory_pool(), io::AsyncContext(thread_pool.get()), input_stream,

Review comment:
       I can open a JIRA once we settle on what the desired behavior is but your second comment has left me unsure.  Are we saying that the method shouldn't accept an AsyncContext at all?  Otherwise, if it does, how should that be interpreted?
   
   Do filesystems get to own their executors?  Or do caller's need to specify that in addition to the CPU executor?  Should these be specified every call?  Or should executors be somewhat "global" and specified through configuration (similar to how the default memory pool is set).




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565970999



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);

Review comment:
       Done

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;

Review comment:
       Done




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



[GitHub] [arrow] nealrichardson commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
nealrichardson commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r570465569



##########
File path: cpp/src/arrow/csv/options.h
##########
@@ -119,6 +119,13 @@ struct ARROW_EXPORT ReadOptions {
 
   /// Whether to use the global CPU thread pool
   bool use_threads = true;
+  /// TODO: This will come out before this gets merged in.  There is not much point in

Review comment:
       Ready to remove this?




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565983607



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.
+template <typename T>
+class BackgroundIterator {
+  using PromiseType = typename detail::BackgroundIteratorPromise<T>;
+
+ public:
+  explicit BackgroundIterator(Iterator<T> it, internal::Executor* executor)
+      : it_(new Iterator<T>(std::move(it))),
+        queue_(new detail::ReadaheadQueue(0)),
+        executor_(executor),
+        done_() {}
+
+  ~BackgroundIterator() {
+    if (queue_) {
+      // Make sure the queue doesn't call any promises after this object
+      // is destroyed.
+      queue_->EnsureShutdownOrDie();
+    }
+  }
+
+  ARROW_DEFAULT_MOVE_AND_ASSIGN(BackgroundIterator);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(BackgroundIterator);
+
+  Future<T> operator()() {
+    if (done_) {
+      return Future<T>::MakeFinished(IterationTraits<T>::End());
+    }
+    auto promise = std::unique_ptr<PromiseType>(new PromiseType{it_.get()});
+    auto result = Future<T>(promise->out_);

Review comment:
       Renamed to `future`.  I'll have to break my habit of always calling the "thing I'm going to return" `result`.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565959510



##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,9 +832,203 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(3, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(Status::IOError("XYZ"), results[1].status());
+    ASSERT_EQ(3, *results[2]);
+  }));
+
+  f1.MarkFinished(1);
+  f2.MarkFinished(Status::IOError("XYZ"));
+  f3.MarkFinished(3);
+
+  AssertFinished(combined);
+}
+
+TEST(FutureLoopTest, Sync) {
+  struct {
+    int i = 0;
+    Future<int> Get() { return Future<int>::MakeFinished(i++); }
+  } IntSource;
+
+  bool do_fail = false;
+  std::vector<int> ints;
+  auto loop_body = [&] {
+    return IntSource.Get().Then([&](int i) -> Result<ControlFlow<int>> {
+      if (do_fail && i == 3) {
+        return Status::IOError("xxx");
+      }
+
+      if (i == 5) {
+        int sum = 0;
+        for (int i : ints) sum += i;
+        return Break(sum);
+      }
+
+      ints.push_back(i);
+      return Continue();
+    });
+  };
+
+  {
+    auto sum_fut = Loop(loop_body);
+    AssertSuccessful(sum_fut);
+
+    ASSERT_OK_AND_ASSIGN(auto sum, sum_fut.result());
+    ASSERT_EQ(sum, 0 + 1 + 2 + 3 + 4);
+  }
+
+  {
+    do_fail = true;
+    IntSource.i = 0;
+    auto sum_fut = Loop(loop_body);
+    AssertFailed(sum_fut);
+    ASSERT_RAISES(IOError, sum_fut.result());
+  }
+}
+
+TEST(FutureLoopTest, EmptyBreakValue) {
+  Future<> none_fut =
+      Loop([&] { return Future<>::MakeFinished().Then([&](...) { return Break(); }); });
+  AssertSuccessful(none_fut);
+}
+
+TEST(FutureLoopTest, MoveOnlyBreakValue) {
+  Future<MoveOnlyDataType> one_fut = Loop([&] {
+    return Future<int>::MakeFinished(1).Then(
+        [&](int i) { return Break(MoveOnlyDataType(i)); });
+  });
+  AssertSuccessful(one_fut);
+  ASSERT_OK_AND_ASSIGN(auto one, std::move(one_fut).result());
+  ASSERT_EQ(one, 1);
+}
+
+TEST(FutureLoopTest, StackOverflow) {
+  // Looping over futures is normally a rather recursive task.  If the futures complete
+  // synchronously (because they are already finished) it could lead to a stack overflow
+  // if care is not taken.
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 1000000) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());
+    }
+    return Future<ControlFlow<int>>::MakeFinished(Break(-1));
+  };
+  auto loop_fut = Loop(loop_body);
+  ASSERT_TRUE(loop_fut.Wait(0.1));
+}
+
+TEST(FutureLoopTest, AllowsBreakFutToBeDiscarded) {
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 10) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());
+    }
+    return Future<ControlFlow<int>>::MakeFinished(Break(-1));
+  };
+  auto loop_fut = Loop(loop_body).Then([](...) { return Status::OK(); });
+  ASSERT_TRUE(loop_fut.Wait(0.1));
+}
+
+TEST(FutureLoopTest, EmptyLoop) {
+  auto loop_body = []() -> Future<ControlFlow<int>> {
+    return Future<ControlFlow<int>>::MakeFinished(Break(0));
+  };
+  auto loop_fut = Loop(loop_body);
+  ASSERT_TRUE(loop_fut.Wait(0.1));
+  if (loop_fut.is_finished()) {

Review comment:
       You are correct.  I had noticed that some gtest clauses did not abort the function but I hadn't realized the subtleties.  I changed this to ASSERT_FINISHES_OK_AND_ASSIGN.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564857557



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}
+
+TEST(TestAsyncIteratorTransform, SkipSome) {
+  auto original = AsyncVectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = TransformAsyncGenerator(std::move(original), filter);
+  AssertAsyncGeneratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestAsyncUtil, ReadaheadFailed) {
+  auto source = []() -> Future<TestInt> {
+    return Future<TestInt>::MakeFinished(Status::Invalid("X"));
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  auto next = readahead();
+  ASSERT_EQ(Status::Invalid("X"), next.status());
+}
+
+TEST(TestAsyncUtil, Readahead) {
+  int num_delivered = 0;
+  auto source = [&num_delivered]() {
+    if (num_delivered < 5) {
+      return Future<TestInt>::MakeFinished(num_delivered++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  // Should not pump until first item requested

Review comment:
       This is indeed different than the synchronous implementation.  I don't think the synchronous implementation is correct.  For example, if I create a `ThreadedTableReader` then it will start reading from the source file BEFORE I call `reader->Read()`.  I find this odd and not the desired behavior.
   
   The AsyncThreadedTableReader will not start reading from the file until the caller calls `reader->Read()` but it will create the background thread.  Once we make the filesystem interfaces async then ideally it won't even create a background thread or do any work other than instantiation.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565969645



##########
File path: cpp/src/arrow/util/thread_pool.h
##########
@@ -87,6 +87,21 @@ class ARROW_EXPORT Executor {
     return SpawnReal(hints, std::forward<Function>(func));
   }
 
+  template <typename T>
+  Future<T> Transfer(Future<T> future) {
+    auto transferred = Future<T>::Make();
+    future.AddCallback([this, transferred](const Result<T>& result) mutable {
+      Result<T> result_copy(result);
+      auto spawn_status = Spawn([transferred, result_copy]() mutable {
+        transferred.MarkFinished(result_copy);

Review comment:
       This changed to a move.




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



[GitHub] [arrow] pitrou commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-775245701


   If I use the CPU thread pool for `TaskGroup::MakeThreaded`, I see no regressions between this and git master on a bunch of macro benchmarks (various real-world CSV files gathered from the Internet). Nice :-)


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565937536



##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NTASKS = 100;
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+
+  std::vector<Future<std::shared_ptr<Table>>> task_futures(NTASKS);
+  for (int i = 0; i < NTASKS; i++) {
+    auto input = std::make_shared<io::BufferReader>(table_buffer);
+    ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+    task_futures[i] = reader->ReadAsync();
+  }
+  auto combined_future = All(task_futures);
+  combined_future.Wait();
+
+  ASSERT_OK_AND_ASSIGN(std::vector<Result<std::shared_ptr<Table>>> results,
+                       combined_future.result());
+  for (auto&& result : results) {
+    ASSERT_OK_AND_ASSIGN(auto table, result);
+    ASSERT_EQ(NROWS, table->num_rows());
+  }
+}
+
+void TestNestedParallelism(
+    std::shared_ptr<internal::ThreadPool> thread_pool,
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+  auto input = std::make_shared<io::BufferReader>(table_buffer);
+  ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+
+  Future<std::shared_ptr<Table>> table_future;
+
+  auto read_task = [&reader, &table_future]() mutable {
+    table_future = reader->ReadAsync();
+    return Status::OK();
+  };
+  ASSERT_OK_AND_ASSIGN(auto future, thread_pool->Submit(read_task));
+  ASSERT_TRUE(future.Wait(1));
+
+  if (future.is_finished()) {
+    ASSERT_TRUE(table_future.Wait(1));
+    if (table_future.is_finished()) {

Review comment:
       Replaced with ASSERT_FINISHES_OK_AND_ASSIGN




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565977745



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.
+template <typename T>
+class BackgroundIterator {

Review comment:
       I've changed the name to `BackgroundGenerator`




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



[GitHub] [arrow] ursabot commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-776517687


   ubuntu-20.04-x86_64: https://conbench.ursa.dev/compare/runs/3aa6839d-e95c-4364-8e91-6edad18f0a72...052d04ea-5d6f-48c5-bbfe-0ccd4d70a8b6/
   dgx-ubuntu-18.04-x86_64: https://conbench.ursa.dev/compare/runs/cde6a063-c0f9-4364-8295-8742f5e44649...82b1ac6c-9771-4c30-91f7-a96492df34be/


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r555195471



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {

Review comment:
       I added a comment to MakeTransformedIterator and TransformAsynchronousGenerator.  I also cleaned up the logic in these loops a little.  In fact, `Pump` no longer contains a `while` loop as it was guaranteed to be executed only once anyways.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565952802



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {

Review comment:
       Ok, so I converted it to locally scoped structs.  They are named though...is there some trick to make them fully anonymous or is this what you were thinking?
   
   It removes the copy of the generator.  The copy of the visitor is unavoidable.  It will be added as a callback to multiple futures.  So the visitor will get copied for every future that comes in.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564854967



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}
+
+TEST(TestAsyncIteratorTransform, SkipSome) {
+  auto original = AsyncVectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = TransformAsyncGenerator(std::move(original), filter);
+  AssertAsyncGeneratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestAsyncUtil, ReadaheadFailed) {
+  auto source = []() -> Future<TestInt> {
+    return Future<TestInt>::MakeFinished(Status::Invalid("X"));
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  auto next = readahead();
+  ASSERT_EQ(Status::Invalid("X"), next.status());

Review comment:
       In this case no.  They will all be marked finished with `IterationTraits<TestInt>::End()` I think.  Same as previous comment.  Once an `AsyncGenerator<T>` returns error or `IterationTraits<T>::End()` then it should return `IterationTraits<T>::End()` for all calls to `Next()` made in the future.  Any outstanding returns from `Next()` could resolve successfully, in error, or with the end token.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564862274



##########
File path: cpp/src/arrow/util/task_group.cc
##########
@@ -135,6 +149,18 @@ class ThreadedTaskGroup : public TaskGroup {
       // before cv.notify_one() has returned
       std::unique_lock<std::mutex> lock(mutex_);
       cv_.notify_one();
+      if (completion_future_.has_value()) {
+        // MarkFinished could be slow.  We don't want to call it while we are holding
+        // the lock.
+        // TODO: If optional is thread safe then we can skip this locking entirely
+        auto future = *completion_future_;
+        auto finished = completion_future_->is_finished();
+        auto status = status_;

Review comment:
       Technically we don't set `finished_` when calling `FinishAsync` and so conceivably a task could be scheduled after we call unlock that fails quickly and modifies `status_` as we are using it...but we should probably mark `finished_` when we call `FinishAsync`.  I will do that.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565980902



##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -220,14 +238,36 @@ class ThreadedBlockReader : public BlockReader {
  public:
   using BlockReader::BlockReader;
 
-  Result<arrow::util::optional<CSVBlock>> Next() {
+  static Iterator<util::optional<CSVBlock>> MakeIterator(
+      Iterator<std::shared_ptr<Buffer>> buffer_iterator, std::unique_ptr<Chunker> chunker,
+      std::shared_ptr<Buffer> first_buffer) {
+    auto block_reader =
+        std::make_shared<ThreadedBlockReader>(std::move(chunker), first_buffer);
+    // Wrap shared pointer in callable
+    Transformer<std::shared_ptr<Buffer>, util::optional<CSVBlock>> block_reader_fn =
+        [block_reader](std::shared_ptr<Buffer> next) { return (*block_reader)(next); };
+    return MakeTransformedIterator(std::move(buffer_iterator), block_reader_fn);
+  }
+
+  static AsyncGenerator<util::optional<CSVBlock>> MakeAsyncIterator(
+      AsyncGenerator<std::shared_ptr<Buffer>> buffer_generator,
+      std::unique_ptr<Chunker> chunker, std::shared_ptr<Buffer> first_buffer) {
+    auto block_reader =
+        std::make_shared<ThreadedBlockReader>(std::move(chunker), first_buffer);
+    // Wrap shared pointer in callable
+    Transformer<std::shared_ptr<Buffer>, util::optional<CSVBlock>> block_reader_fn =
+        [block_reader](std::shared_ptr<Buffer> next) { return (*block_reader)(next); };
+    return TransformAsyncGenerator(buffer_generator, block_reader_fn);

Review comment:
       Fixed.




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



[GitHub] [arrow] github-actions[bot] removed a comment on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
github-actions[bot] removed a comment on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-754137607


   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   Could you open an issue for this pull request on JIRA?
   https://issues.apache.org/jira/browse/ARROW
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r574208592



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,382 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;

Review comment:
       I've made async generator consistent for now until someone proposes a better name.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r555259293



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +611,73 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>
+Future<BreakValueType> Loop(Iterate iterate) {

Review comment:
       As long as you don't mind changing Loop to use std::function instead of a template parameter?  It's not a big change but it is different.
   
   I think this fits as you can sort of (implicitly) think of `Break(...) == IterationTraits<ControlFlow<T>>::End()`




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565175198



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +187,122 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_ = false;
+  bool ready_for_next_ = false;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<Result<TransformFlow<V>>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  // Calls the transform function on the current value.  Can return in several ways
+  // * If the next value is requested (e.g. skip) it will return an empty optional
+  // * If an invalid status is encountered that will be returned
+  // * If finished it will return IterationTraits<V>::End()
+  // * If a value is returned by the transformer that will be returned
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      ARROW_ASSIGN_OR_RAISE(util::optional<V> next, Pump());
+      if (next.has_value()) {
+        return *next;
+      }
+      ARROW_ASSIGN_OR_RAISE(last_value_, it_.Next());
+    }
+    return IterationTraits<V>::End();
+  }
+
+ private:
+  Iterator<T> it_;
+  Transformer<T, V> transformer_;
+  util::optional<T> last_value_;
+  bool finished_ = false;
+};
+
+/// \brief Transforms an iterator according to a transformer, returning a new Iterator.
+///
+/// The transformer will be called on each element of the source iterator and for each
+/// call it can yield a value, skip, or finish the iteration.  When yielding a value the
+/// transformer can choose to consume the source item (the default, ready_for_next = true)
+/// or to keep it and it will be called again on the same value.

Review comment:
       I would expect the decompression iterator to do its own internal buffering instead of asking for the same item several times. This is how `CompressedInputStream` already works. The decompression iterator could be factored out from that.
   
   (also see `TransformInputStream` :-))




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565173501



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +615,74 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>
+Future<BreakValueType> Loop(Iterate iterate) {
+  auto break_fut = Future<BreakValueType>::Make();
+
+  struct Callback {
+    bool CheckForTermination(const Result<Control>& maybe_control) {
+      if (!maybe_control.ok() || maybe_control->IsBreak()) {
+        Result<BreakValueType> maybe_break = maybe_control.Map(Control::MoveBreakValue);
+        break_fut.MarkFinished(std::move(maybe_break));
+        return true;
+      }
+      return false;
+    }
+
+    void operator()(const Result<Control>& maybe_control) && {
+      if (CheckForTermination(maybe_control)) return;
+
+      auto control_fut = iterate();
+      while (control_fut.is_finished()) {
+        // There's no need to AddCallback on a finished future; we can CheckForTermination
+        // now. This also avoids recursion and potential stack overflow.
+        if (CheckForTermination(control_fut.result())) return;
+
+        control_fut = iterate();
+      }
+      control_fut.AddCallback(std::move(*this));
+    }
+
+    Iterate iterate;
+    // If the future returned by control_fut is never completed then we will be hanging on
+    // to break_fut forever even if the listener has given up listening on it.  Instead we
+    // rely on the fact that a producer (the caller of Future<>::Make) is always
+    // responsible for completing the futures they create.
+    // TODO: Could avoid this kind of situation with "future abandonment" similar to mesos
+    Future<BreakValueType> break_fut;

Review comment:
       Ok, I see.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r555260518



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {

Review comment:
       I added comments to MakeTransformedIterator and TransformAsyncGenerator




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



[GitHub] [arrow] bkietz commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r556129806



##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,6 +832,145 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(3, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(Status::IOError("XYZ"), results[1].status());
+    ASSERT_EQ(3, *results[2]);
+  }));
+
+  f1.MarkFinished(1);
+  f2.MarkFinished(Status::IOError("XYZ"));
+  f3.MarkFinished(3);
+
+  AssertFinished(combined);
+}
+
+TEST(FutureLoopTest, Sync) {
+  struct {
+    int i = 0;
+    Future<int> Get() { return Future<int>::MakeFinished(i++); }
+  } IntSource;
+
+  bool do_fail = false;
+  std::vector<int> ints;
+  auto loop_body = [&] {
+    return IntSource.Get().Then([&](int i) -> Result<ControlFlow<int>> {
+      if (do_fail && i == 3) {
+        return Status::IOError("xxx");
+      }
+
+      if (i == 5) {
+        int sum = 0;
+        for (int i : ints) sum += i;
+        return Break(sum);
+      }
+
+      ints.push_back(i);
+      return Continue();
+    });
+  };
+
+  {
+    auto sum_fut = Loop(loop_body);
+    AssertSuccessful(sum_fut);
+
+    ASSERT_OK_AND_ASSIGN(auto sum, sum_fut.result());
+    ASSERT_EQ(sum, 0 + 1 + 2 + 3 + 4);
+  }
+
+  {
+    do_fail = true;
+    IntSource.i = 0;
+    auto sum_fut = Loop(loop_body);
+    AssertFailed(sum_fut);
+    ASSERT_RAISES(IOError, sum_fut.result());
+  }
+}
+
+TEST(FutureLoopTest, EmptyBreakValue) {
+  Future<> none_fut =
+      Loop([&] { return Future<>::MakeFinished().Then([&](...) { return Break(); }); });
+  AssertSuccessful(none_fut);
+}
+
+TEST(FutureLoopTest, MoveOnlyBreakValue) {
+  Future<MoveOnlyDataType> one_fut = Loop([&] {
+    return Future<int>::MakeFinished(1).Then(
+        [&](int i) { return Break(MoveOnlyDataType(i)); });
+  });
+  AssertSuccessful(one_fut);
+  ASSERT_OK_AND_ASSIGN(auto one, std::move(one_fut).result());
+  ASSERT_EQ(one, 1);
+}
+
+TEST(FutureLoopTest, StackOverflow) {
+  // Looping over futures is normally a rather recursive task.  If the futures complete
+  // synchronously (because they are already finished) it could lead to a stack overflow
+  // if care is not taken.
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 1000000) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());

Review comment:
       For lambdas without an explicit trailing return type, all return statements *must* return the same type, so you'd have to choose which boilerplate is preferable. IMHO, the trailing return type and implicit conversion look prettiest




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565778014



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      util::optional<Result<V>> next = Pump();
+      if (next.has_value()) {
+        return *next;

Review comment:
       Is this necessary?  Isn't there an implicit move on return values?  I need more guidance on when it is neccesary to move return values.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565987394



##########
File path: cpp/src/arrow/util/iterator.cc
##########
@@ -119,14 +123,30 @@ class ReadaheadQueue::Impl : public std::enable_shared_from_this<ReadaheadQueue:
   void DoWork() {
     std::unique_lock<std::mutex> lock(mutex_);
     while (!please_shutdown_) {
-      while (static_cast<int64_t>(done_.size()) < max_readahead_ && todo_.size() > 0) {
+      while (todo_.size() > 0 &&
+             ((max_readahead_ <= 0) ||
+              (static_cast<int64_t>(done_.size()) < max_readahead_))) {
         auto promise = std::move(todo_.front());
         todo_.pop_front();
         lock.unlock();
-        promise->Call();
+        if (promise->Call()) {
+          // If the call finished then we should purge the remaining TODO items, marking
+          // them finished
+          lock.lock();
+          std::deque<std::unique_ptr<ReadaheadPromise>> to_clear(std::move(todo_));
+          // While the async iterator doesn't use todo_ anymore after it hits a finish the
+          // sync iterator might still due to timing so leave it valid
+          todo_.clear();
+          lock.unlock();
+          for (auto&& promise : to_clear) {
+            promise->End();
+          }
+        }
         lock.lock();
-        done_.push_back(std::move(promise));
-        work_done_.notify_one();
+        if (max_readahead_ > 0) {
+          done_.push_back(std::move(promise));
+          work_done_.notify_one();
+        }

Review comment:
       Also, it isn't "unbounded readahead".  Both implementations right now have two queues.  In the synchronous case both queues are in ReadaheadQueueImpl (`todo_` and `done_`).  In the asynchronous case there is one queue in ReadaheadQueueImpl (`todo_`) and the other is in `ReadaheadGenerator (`readahead_queue_`).  The asynchronous case enforces the queue length with `readahead_queue_` and the background thread will block if `readahead_queue_` isn't pulled fast enough (because nothing will be inserted in `todo_`).
   
   Now, I can also understand how that isn't clear at all in the code.  I still plan on taking a pass at simplifying things.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564825292



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +615,74 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>
+Future<BreakValueType> Loop(Iterate iterate) {
+  auto break_fut = Future<BreakValueType>::Make();
+
+  struct Callback {
+    bool CheckForTermination(const Result<Control>& maybe_control) {
+      if (!maybe_control.ok() || maybe_control->IsBreak()) {
+        Result<BreakValueType> maybe_break = maybe_control.Map(Control::MoveBreakValue);
+        break_fut.MarkFinished(std::move(maybe_break));
+        return true;
+      }
+      return false;
+    }
+
+    void operator()(const Result<Control>& maybe_control) && {
+      if (CheckForTermination(maybe_control)) return;
+
+      auto control_fut = iterate();
+      while (control_fut.is_finished()) {
+        // There's no need to AddCallback on a finished future; we can CheckForTermination
+        // now. This also avoids recursion and potential stack overflow.

Review comment:
       There is a mutex inside of `AddCallback`.  We could create a `TryAddCallback` which returns false without running the callback if the future is finished.  This method could be guaranteed to never run the callback synchronously.  I will add this.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565103153



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.

Review comment:
       No, not in an executor at the moment, and probably never will.  At some point, I think the filesystem interfaces will support async and then the filesystems themselves can assign executors where appropriate.  Once that is done there will be no need for this "backgrounding" class.  The background threads certainly don't belong on the CPU thread pool.  We could create a "background" thread pool but it would have to be large enough to support the maximum parallelism we want from the filesystem (which may be greater than the # of CPUs in a slow networked I/O case).
   
   This generator (I will change that name per next comment) can indeed be called from several threads at once.  Although there is no need for it and we can do away with that requirement if it helps us save on concurrency cost.  I'll write a separate note on the parallelism of these generators.  The underlying iterator will never be iterated on by any more than the background thread.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565978976



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.
+template <typename T>
+class BackgroundIterator {
+  using PromiseType = typename detail::BackgroundIteratorPromise<T>;
+
+ public:
+  explicit BackgroundIterator(Iterator<T> it, internal::Executor* executor)
+      : it_(new Iterator<T>(std::move(it))),
+        queue_(new detail::ReadaheadQueue(0)),
+        executor_(executor),
+        done_() {}
+
+  ~BackgroundIterator() {
+    if (queue_) {
+      // Make sure the queue doesn't call any promises after this object
+      // is destroyed.
+      queue_->EnsureShutdownOrDie();
+    }
+  }
+
+  ARROW_DEFAULT_MOVE_AND_ASSIGN(BackgroundIterator);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(BackgroundIterator);
+
+  Future<T> operator()() {
+    if (done_) {
+      return Future<T>::MakeFinished(IterationTraits<T>::End());
+    }
+    auto promise = std::unique_ptr<PromiseType>(new PromiseType{it_.get()});
+    auto result = Future<T>(promise->out_);
+    // TODO: Need a futuristic version of ARROW_RETURN_NOT_OK
+    auto append_status = queue_->Append(
+        static_cast<std::unique_ptr<detail::ReadaheadPromise>>(std::move(promise)));
+    if (!append_status.ok()) {
+      return Future<T>::MakeFinished(append_status);
+    }
+
+    result.AddCallback([this](const Result<T>& result) {
+      if (!result.ok() || result.ValueUnsafe() == IterationTraits<T>::End()) {
+        done_ = true;
+      }
+    });
+
+    return executor_->Transfer(result);

Review comment:
       Actually, that trivial callback added a few lines above will not run on the executor.  It will run on the I/O background thread.  Instead any continuation added (by the caller) to the returned future will be run on the executor.  This is how we transfer control from the background I/O thread to the CPU executor.




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



[GitHub] [arrow] ursabot commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-772101522


   ubuntu-20.04-x86_64: https://conbench.ursa.dev/compare/runs/9201d7f7-4ecd-469f-8300-b8c3addf63c0...59fa7d42-df11-4f93-a6fb-e602926831aa/


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564828137



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +187,122 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_ = false;
+  bool ready_for_next_ = false;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {

Review comment:
       Needed for the "called again on the same value" case.  More in next comment.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564820568



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +615,74 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>
+Future<BreakValueType> Loop(Iterate iterate) {
+  auto break_fut = Future<BreakValueType>::Make();
+
+  struct Callback {
+    bool CheckForTermination(const Result<Control>& maybe_control) {
+      if (!maybe_control.ok() || maybe_control->IsBreak()) {
+        Result<BreakValueType> maybe_break = maybe_control.Map(Control::MoveBreakValue);
+        break_fut.MarkFinished(std::move(maybe_break));
+        return true;
+      }
+      return false;
+    }
+
+    void operator()(const Result<Control>& maybe_control) && {
+      if (CheckForTermination(maybe_control)) return;
+
+      auto control_fut = iterate();
+      while (control_fut.is_finished()) {
+        // There's no need to AddCallback on a finished future; we can CheckForTermination
+        // now. This also avoids recursion and potential stack overflow.
+        if (CheckForTermination(control_fut.result())) return;
+
+        control_fut = iterate();
+      }
+      control_fut.AddCallback(std::move(*this));
+    }
+
+    Iterate iterate;
+    // If the future returned by control_fut is never completed then we will be hanging on
+    // to break_fut forever even if the listener has given up listening on it.  Instead we
+    // rely on the fact that a producer (the caller of Future<>::Make) is always
+    // responsible for completing the futures they create.
+    // TODO: Could avoid this kind of situation with "future abandonment" similar to mesos
+    Future<BreakValueType> break_fut;

Review comment:
       No.  It is quite normal for the future consumer to abandon it.  For example...
   
   ```
   AsyncVisit(...).Then(PrintSomething);
   ```
   Two futures are created.  Neither one is kept.  Also, the future created by the call to `Then` does not keep a strong reference to its predecessor.  Even if we add such a reference it doesn't really help the situation.  If we make the reference a `WeakFuture` then the future would immediately go out of scope and be destroyed.  
   
   The true owner of the future is the producer.  If the future producer never completes the future there will be a memory leak.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564743436



##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -177,14 +182,26 @@ class SerialBlockReader : public BlockReader {
  public:
   using BlockReader::BlockReader;
 
-  Result<arrow::util::optional<CSVBlock>> Next() {
+  static Iterator<util::optional<CSVBlock>> MakeIterator(
+      Iterator<std::shared_ptr<Buffer>> buffer_iterator, std::unique_ptr<Chunker> chunker,
+      std::shared_ptr<Buffer> first_buffer) {
+    auto block_reader =
+        std::make_shared<SerialBlockReader>(std::move(chunker), first_buffer);
+    // Wrap shared pointer in callable
+    Transformer<std::shared_ptr<Buffer>, util::optional<CSVBlock>> block_reader_fn =
+        [block_reader](std::shared_ptr<Buffer> buf) {
+          return (*block_reader)(std::move(buf));
+        };
+    return MakeTransformedIterator(std::move(buffer_iterator), block_reader_fn);
+  }
+
+  Result<TransformFlow<util::optional<CSVBlock>>> operator()(

Review comment:
       `TransformFinish` (line 201) will cause `IterationTraits<util::optional<CSVBlock>>::End()` to be emitted.  So it was either keeping the `util::optional` or adding `IterationTraits<CSVBlock>::End()` (and a corresponding equality operator).
   
   The async iterators are still iterators and so they rely on that end token.  One thing we could do is rewire all the async iterator functions so that they use `util::optional` under the hood (unless the type being iterated is a pointer) and it doesn't allow users to specify their own end tokens.  Then the `util::optional` could be hidden from the consumer of the iterators API.  This would require all users to rely on `Visit` instead of manually iterating.  However, for async iterator, this is probably a given already.




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r566307246



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {

Review comment:
       > is there some trick to make them fully anonymous or is this what you were thinking?
   
   No, this is what I was thinking (bad terminology, sorry :-)).




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r572272932



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,382 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;

Review comment:
       Note that Python now has asynchronous iterators [as well](https://docs.python.org/3/glossary.html#term-asynchronous-iterator).
   (and you can implement a Python asynchronous iterator as an asynchronous generator ;-))




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



[GitHub] [arrow] ursabot commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-777148448


   ubuntu-20.04-x86_64: https://conbench.ursa.dev/compare/runs/a4551972-3bf9-4ba7-bef4-ec9ed683c09a...9cd3bf23-4fa0-4c1a-a61f-75d7171e59a7/
   dgx-ubuntu-18.04-x86_64: https://conbench.ursa.dev/compare/runs/47769121-ed42-4698-89fe-3d851f2df615...3e7a0ef7-ed09-4b44-891d-7aa9ca66dad6/


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



[GitHub] [arrow] westonpace commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-771930077


   @ursabot please 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565969245



##########
File path: cpp/src/arrow/util/task_group.cc
##########
@@ -135,6 +149,18 @@ class ThreadedTaskGroup : public TaskGroup {
       // before cv.notify_one() has returned
       std::unique_lock<std::mutex> lock(mutex_);
       cv_.notify_one();
+      if (completion_future_.has_value()) {
+        // MarkFinished could be slow.  We don't want to call it while we are holding
+        // the lock.
+        // TODO: If optional is thread safe then we can skip this locking entirely
+        auto future = *completion_future_;
+        auto finished = completion_future_->is_finished();
+        auto status = status_;
+        lock.unlock();
+        if (!finished) {
+          future.MarkFinished(status);

Review comment:
       I struggled to create a test case that could reproduce this. It seems like it should be reproducible but even adding a slow callback (so MarkFinished took a long time) it was difficult to get a task to be added and then finish just when I wanted it to. I have added some logic however so that this race condition should not be possible any longer.

##########
File path: cpp/src/arrow/util/thread_pool.h
##########
@@ -87,6 +87,21 @@ class ARROW_EXPORT Executor {
     return SpawnReal(hints, std::forward<Function>(func));
   }
 
+  template <typename T>
+  Future<T> Transfer(Future<T> future) {
+    auto transferred = Future<T>::Make();
+    future.AddCallback([this, transferred](const Result<T>& result) mutable {
+      Result<T> result_copy(result);

Review comment:
       This copy is removed completely.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565967704



##########
File path: cpp/src/arrow/util/task_group.cc
##########
@@ -135,6 +149,18 @@ class ThreadedTaskGroup : public TaskGroup {
       // before cv.notify_one() has returned
       std::unique_lock<std::mutex> lock(mutex_);
       cv_.notify_one();
+      if (completion_future_.has_value()) {
+        // MarkFinished could be slow.  We don't want to call it while we are holding
+        // the lock.
+        // TODO: If optional is thread safe then we can skip this locking entirely
+        auto future = *completion_future_;

Review comment:
       Done




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565931217



##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,6 +832,145 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(3, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(Status::IOError("XYZ"), results[1].status());
+    ASSERT_EQ(3, *results[2]);
+  }));
+
+  f1.MarkFinished(1);
+  f2.MarkFinished(Status::IOError("XYZ"));
+  f3.MarkFinished(3);
+
+  AssertFinished(combined);
+}
+
+TEST(FutureLoopTest, Sync) {
+  struct {
+    int i = 0;
+    Future<int> Get() { return Future<int>::MakeFinished(i++); }
+  } IntSource;
+
+  bool do_fail = false;
+  std::vector<int> ints;
+  auto loop_body = [&] {
+    return IntSource.Get().Then([&](int i) -> Result<ControlFlow<int>> {
+      if (do_fail && i == 3) {
+        return Status::IOError("xxx");
+      }
+
+      if (i == 5) {
+        int sum = 0;
+        for (int i : ints) sum += i;
+        return Break(sum);
+      }
+
+      ints.push_back(i);
+      return Continue();
+    });
+  };
+
+  {
+    auto sum_fut = Loop(loop_body);
+    AssertSuccessful(sum_fut);
+
+    ASSERT_OK_AND_ASSIGN(auto sum, sum_fut.result());
+    ASSERT_EQ(sum, 0 + 1 + 2 + 3 + 4);
+  }
+
+  {
+    do_fail = true;
+    IntSource.i = 0;
+    auto sum_fut = Loop(loop_body);
+    AssertFailed(sum_fut);
+    ASSERT_RAISES(IOError, sum_fut.result());
+  }
+}
+
+TEST(FutureLoopTest, EmptyBreakValue) {
+  Future<> none_fut =
+      Loop([&] { return Future<>::MakeFinished().Then([&](...) { return Break(); }); });
+  AssertSuccessful(none_fut);
+}
+
+TEST(FutureLoopTest, MoveOnlyBreakValue) {

Review comment:
       I'm not sure what to stress.  The Loop call is made once and returns a future.  It can't be called by multiple threads, each call would construct an entirely separate loop.  It iterates in a serial (non reentrant pulling) fashion through the generator.  There is a stack overflow test I have added to test the stack overflow case.  Other than that I'm not sure what stressful case to add.




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r572150949



##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -837,20 +885,121 @@ class ThreadedTableReader : public BaseTableReader {
   }
 
  protected:
-  ThreadPool* thread_pool_;
+  Executor* thread_pool_;
+  Iterator<std::shared_ptr<Buffer>> buffer_iterator_;
+};
+
+class AsyncThreadedTableReader
+    : public BaseTableReader,
+      public std::enable_shared_from_this<AsyncThreadedTableReader> {
+ public:
+  using BaseTableReader::BaseTableReader;
+
+  AsyncThreadedTableReader(MemoryPool* pool, std::shared_ptr<io::InputStream> input,
+                           const ReadOptions& read_options,
+                           const ParseOptions& parse_options,
+                           const ConvertOptions& convert_options, Executor* thread_pool)
+      : BaseTableReader(pool, input, read_options, parse_options, convert_options),
+        thread_pool_(thread_pool) {}
+
+  ~AsyncThreadedTableReader() override {
+    if (task_group_) {
+      // In case of error, make sure all pending tasks are finished before
+      // we start destroying BaseTableReader members
+      ARROW_UNUSED(task_group_->Finish());
+    }
+  }
+
+  Status Init() override {
+    ARROW_ASSIGN_OR_RAISE(auto istream_it,
+                          io::MakeInputStreamIterator(input_, read_options_.block_size));
+
+    ARROW_ASSIGN_OR_RAISE(auto bg_it,
+                          MakeBackgroundIterator(std::move(istream_it), thread_pool_));
+
+    int32_t block_queue_size = thread_pool_->GetCapacity();
+    auto rh_it = AddReadahead(bg_it, block_queue_size);
+    buffer_generator_ = CSVBufferIterator::MakeAsync(std::move(rh_it));
+    return Status::OK();
+  }
+
+  Result<std::shared_ptr<Table>> Read() override { return ReadAsync().result(); }
+
+  Future<std::shared_ptr<Table>> ReadAsync() override {
+    task_group_ = internal::TaskGroup::MakeThreaded(thread_pool_);

Review comment:
       Ok, I just noticed that `internal::TaskGroup::MakeThreaded(thread_pool_)` isn't good. It will transfer CPU-intensive tasks on the same thread pool that's used for IO. Also, it gets a fixed size of 8 threads, even on my 24-thread CPU.




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



[GitHub] [arrow] bkietz commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r562883545



##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -150,20 +154,21 @@ struct CSVBlock {
   std::function<Status(int64_t)> consume_bytes;
 };
 
+// This is an unfortunate side-effect of using optional<T> as the iterator in the
+// CSVBlock iterator.  We need to be able to compare with
+// IterationTraits<optional<T>>::End() and empty optionals will always compare true but
+// the optional copmarator won't compile if the underlying type isn't comparable
+bool operator==(const CSVBlock& left, const CSVBlock& right) { return false; }

Review comment:
       Could we just have 
   ```suggestion
   bool operator==(const CSVBlock& left, const CSVBlock& right) { return left.block_index == right.block_index; }
   ```
   ? It'd still be worthwhile to explain that equality comparability is part of the contract of `Iterator`.

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -150,20 +154,21 @@ struct CSVBlock {
   std::function<Status(int64_t)> consume_bytes;
 };
 
+// This is an unfortunate side-effect of using optional<T> as the iterator in the
+// CSVBlock iterator.  We need to be able to compare with
+// IterationTraits<optional<T>>::End() and empty optionals will always compare true but
+// the optional copmarator won't compile if the underlying type isn't comparable
+bool operator==(const CSVBlock& left, const CSVBlock& right) { return false; }

Review comment:
       You could additionally skip `optional` entirely if desired by defining `IterationTraits<CSVBlock>::End()` to have block_index == -1 or so. 




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



[GitHub] [arrow] ursabot commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-777193131


   ubuntu-20.04-x86_64: https://conbench.ursa.dev/compare/runs/a4551972-3bf9-4ba7-bef4-ec9ed683c09a...a48ff071-cd1c-40d9-a936-a8e8c26d494e/
   dgx-ubuntu-18.04-x86_64: https://conbench.ursa.dev/compare/runs/47769121-ed42-4698-89fe-3d851f2df615...06d24f59-3ad7-422f-afe1-c0da38a1ca86/


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565964716



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {

Review comment:
       I reordered the tests to clean this up and will take more care in the future.

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}

Review comment:
       I now test this case.

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}
+
+TEST(TestAsyncIteratorTransform, SkipSome) {
+  auto original = AsyncVectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = TransformAsyncGenerator(std::move(original), filter);
+  AssertAsyncGeneratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestAsyncUtil, ReadaheadFailed) {
+  auto source = []() -> Future<TestInt> {
+    return Future<TestInt>::MakeFinished(Status::Invalid("X"));
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  auto next = readahead();
+  ASSERT_EQ(Status::Invalid("X"), next.status());

Review comment:
       I now test this case.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565970550



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();

Review comment:
       Done




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564802321



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"

Review comment:
       It is needed for the call to executor->Transfer on line 273.  I can't add `Transfer(Executor*)` to `Future` because `Future` is unaware of `Executor` (presumably because `Executor` returns `Future`).




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



[GitHub] [arrow] bkietz commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r554155616



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -35,6 +36,13 @@
 
 namespace arrow {
 
+namespace detail {
+
+template <typename Signature>
+using result_of_t = typename std::result_of<Signature>::type;

Review comment:
       It's technically legal to have this both here and in future.h, but for clarity we should probably have a single alias decl. util/functional.h would be a reasonable place to put it

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      util::optional<Result<V>> next = Pump();
+      if (next.has_value()) {
+        return *next;
+      }
+      ARROW_ASSIGN_OR_RAISE(last_value_, it_.Next());
+    }
+    return IterationTraits<V>::End();
+  }
+
+ private:
+  Iterator<T> it_;
+  Transformer<T, V> transformer_;
+  util::optional<T> last_value_;
+  bool finished_;

Review comment:
       ```suggestion
     bool finished_ = false;
   ```

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {

Review comment:
       This loop is very confusing. Could you rewrite it with a singular condition (`while (!finished_)`, maybe) then include a break statement below?

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      util::optional<Result<V>> next = Pump();
+      if (next.has_value()) {
+        return *next;

Review comment:
       ```suggestion
           return std::move(*next);
   ```

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +611,73 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>
+Future<BreakValueType> Loop(Iterate iterate) {

Review comment:
       Maybe the typedef for AsyncGenerator would be useful here; then we could write
   
   ```suggestion
   template <typename T>
   using AsyncGenerator = std::function<Future<T>()>;
   
   template <typename BreakValue>
   Future<BreakValue> Loop(AsyncGenerator<ControlFlow<BreakValue>> iterate) {
   ```

##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,6 +832,145 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(3, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(Status::IOError("XYZ"), results[1].status());
+    ASSERT_EQ(3, *results[2]);
+  }));
+
+  f1.MarkFinished(1);
+  f2.MarkFinished(Status::IOError("XYZ"));
+  f3.MarkFinished(3);
+
+  AssertFinished(combined);
+}
+
+TEST(FutureLoopTest, Sync) {
+  struct {
+    int i = 0;
+    Future<int> Get() { return Future<int>::MakeFinished(i++); }
+  } IntSource;
+
+  bool do_fail = false;
+  std::vector<int> ints;
+  auto loop_body = [&] {
+    return IntSource.Get().Then([&](int i) -> Result<ControlFlow<int>> {
+      if (do_fail && i == 3) {
+        return Status::IOError("xxx");
+      }
+
+      if (i == 5) {
+        int sum = 0;
+        for (int i : ints) sum += i;
+        return Break(sum);
+      }
+
+      ints.push_back(i);
+      return Continue();
+    });
+  };
+
+  {
+    auto sum_fut = Loop(loop_body);
+    AssertSuccessful(sum_fut);
+
+    ASSERT_OK_AND_ASSIGN(auto sum, sum_fut.result());
+    ASSERT_EQ(sum, 0 + 1 + 2 + 3 + 4);
+  }
+
+  {
+    do_fail = true;
+    IntSource.i = 0;
+    auto sum_fut = Loop(loop_body);
+    AssertFailed(sum_fut);
+    ASSERT_RAISES(IOError, sum_fut.result());
+  }
+}
+
+TEST(FutureLoopTest, EmptyBreakValue) {
+  Future<> none_fut =
+      Loop([&] { return Future<>::MakeFinished().Then([&](...) { return Break(); }); });
+  AssertSuccessful(none_fut);
+}
+
+TEST(FutureLoopTest, MoveOnlyBreakValue) {

Review comment:
       It'd be good to have a stress test for `Loop` as well

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {

Review comment:
       Could you also include a comment describing the control flow, the contract of transformer functions, ...?

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}

Review comment:
       Instead of having TransformFlow potentially be an error, perhaps we could just rely on Result and have `using Transformer = std::function<Result<TransformFlow<V>>(T)>;`?

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {

Review comment:
       I'd usually expect `Result<>` to be outermost in a return type:
   ```suggestion
     Result<util::optional<V>> Pump() {
   ```
   Additionally, that'd enable you to use `ARROW_RETURN_NOT_OK` below

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;

Review comment:
       Use member initializers for fields with no default constructor:
   ```suggestion
     bool finished_ = false;
     bool ready_for_next_ = false;
   ```

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();

Review comment:
       This doesn't match the return type of the function, please use `return {};` or `return util::nullopt;` to be more clear

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -556,6 +560,33 @@ inline bool WaitForAll(const std::vector<Future<T>*>& futures,
   return waiter->Wait(seconds);
 }
 
+template <typename T>

Review comment:
       ```suggestion
   /// \brief Create a Future which completes when all of `futures` complete.
   ///
   /// The future's result is a vector of the results of `futures`.
   /// Note that this future will never be marked "failed"; failed results
   /// will be stored in the result vector alongside successful results.
   template <typename T>
   ```

##########
File path: cpp/src/arrow/util/iterator.cc
##########
@@ -67,6 +67,8 @@ class ReadaheadQueue::Impl : public std::enable_shared_from_this<ReadaheadQueue:
   }
 
   Status PopDone(std::unique_ptr<ReadaheadPromise>* out) {
+    DCHECK_GT(max_readahead_, 0);  // This function has no purpose and should not be
+                                   // called if using the queue unbounded

Review comment:
       `DCHECK` macros (except `DCHECK_OK`) can be streamed into, which makes assert failures more helpful and searchable:
   ```suggestion
       DCHECK_GT(max_readahead_, 0) << "PopDone should never be called if using the queue unbounded";
   ```

##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,6 +832,145 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(3, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(Status::IOError("XYZ"), results[1].status());
+    ASSERT_EQ(3, *results[2]);
+  }));
+
+  f1.MarkFinished(1);
+  f2.MarkFinished(Status::IOError("XYZ"));
+  f3.MarkFinished(3);
+
+  AssertFinished(combined);
+}
+
+TEST(FutureLoopTest, Sync) {
+  struct {
+    int i = 0;
+    Future<int> Get() { return Future<int>::MakeFinished(i++); }
+  } IntSource;
+
+  bool do_fail = false;
+  std::vector<int> ints;
+  auto loop_body = [&] {
+    return IntSource.Get().Then([&](int i) -> Result<ControlFlow<int>> {
+      if (do_fail && i == 3) {
+        return Status::IOError("xxx");
+      }
+
+      if (i == 5) {
+        int sum = 0;
+        for (int i : ints) sum += i;
+        return Break(sum);
+      }
+
+      ints.push_back(i);
+      return Continue();
+    });
+  };
+
+  {
+    auto sum_fut = Loop(loop_body);
+    AssertSuccessful(sum_fut);
+
+    ASSERT_OK_AND_ASSIGN(auto sum, sum_fut.result());
+    ASSERT_EQ(sum, 0 + 1 + 2 + 3 + 4);
+  }
+
+  {
+    do_fail = true;
+    IntSource.i = 0;
+    auto sum_fut = Loop(loop_body);
+    AssertFailed(sum_fut);
+    ASSERT_RAISES(IOError, sum_fut.result());
+  }
+}
+
+TEST(FutureLoopTest, EmptyBreakValue) {
+  Future<> none_fut =
+      Loop([&] { return Future<>::MakeFinished().Then([&](...) { return Break(); }); });
+  AssertSuccessful(none_fut);
+}
+
+TEST(FutureLoopTest, MoveOnlyBreakValue) {
+  Future<MoveOnlyDataType> one_fut = Loop([&] {
+    return Future<int>::MakeFinished(1).Then(
+        [&](int i) { return Break(MoveOnlyDataType(i)); });
+  });
+  AssertSuccessful(one_fut);
+  ASSERT_OK_AND_ASSIGN(auto one, std::move(one_fut).result());
+  ASSERT_EQ(one, 1);
+}
+
+TEST(FutureLoopTest, StackOverflow) {
+  // Looping over futures is normally a rather recursive task.  If the futures complete
+  // synchronously (because they are already finished) it could lead to a stack overflow
+  // if care is not taken.
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 1000000) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());

Review comment:
       Maybe it'd be useful to have an implicit constructor `Future<T>(Result<T>)` for finished futures, then I think we'd be able to just write
   ```c++
         return Continue();
   ```

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +611,73 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,

Review comment:
       ```suggestion
   /// \brief Loop through an asynchronous sequence
   ///
   /// \param[in] iterate A generator of Future<ControlFlow<BreakValue>>. On completion of each yielded
   /// future the resulting ControlFlow will be examined. A Break will terminate the loop, while a Continue
   /// will re-invoke `iterate`.
   /// \return A future which will complete when a Future returned by iterate completes with a Break
   template <typename Iterate,
   ```

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -129,11 +130,47 @@ template <typename T>
 inline Iterator<T> EmptyIt() {
   return MakeEmptyIterator<T>();
 }
-
 inline Iterator<TestInt> VectorIt(std::vector<TestInt> v) {
   return MakeVectorIterator<TestInt>(std::move(v));
 }
 
+std::function<Future<TestInt>()> AsyncVectorIt(std::vector<TestInt> v) {
+  auto index = std::make_shared<size_t>(0);
+  auto vec = std::make_shared<std::vector<TestInt>>(std::move(v));
+  return [index, vec]() -> Future<TestInt> {
+    if (*index >= vec->size()) {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+    auto next = (*vec)[*index];
+    (*index)++;
+    return Future<TestInt>::MakeFinished(next);
+  };

Review comment:
       ```suggestion
     size_t index = 0;
     return [index, v]() mutable -> Future<TestInt> {
       if (index >= vec.size()) {
         return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
       }
       return Future<TestInt>::MakeFinished(v[index++]);
     };
   ```




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



[GitHub] [arrow] ursabot commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-772101522


   ubuntu-20.04-x86_64: https://conbench.ursa.dev/compare/runs/9201d7f7-4ecd-469f-8300-b8c3addf63c0...59fa7d42-df11-4f93-a6fb-e602926831aa/


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564862995



##########
File path: cpp/src/arrow/util/task_group.cc
##########
@@ -135,6 +149,18 @@ class ThreadedTaskGroup : public TaskGroup {
       // before cv.notify_one() has returned
       std::unique_lock<std::mutex> lock(mutex_);
       cv_.notify_one();
+      if (completion_future_.has_value()) {
+        // MarkFinished could be slow.  We don't want to call it while we are holding
+        // the lock.
+        // TODO: If optional is thread safe then we can skip this locking entirely
+        auto future = *completion_future_;
+        auto finished = completion_future_->is_finished();
+        auto status = status_;
+        lock.unlock();
+        if (!finished) {
+          future.MarkFinished(status);

Review comment:
       Indeed.  I will make a test case to expose this.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564736879



##########
File path: cpp/src/arrow/csv/reader_benchmark.cc
##########
@@ -0,0 +1,139 @@
+// 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 <string>
+
+#include "arrow/buffer.h"
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+class SlowInputStream : public io::InputStream {
+ public:
+  explicit SlowInputStream(std::shared_ptr<io::BufferReader> target, int64_t latency_ms)
+      : target_(std::move(target)) {
+    latency_s_ = static_cast<double>(latency_ms) / 1000.0;
+  }
+  virtual ~SlowInputStream() {}
+
+  Result<util::string_view> Peek(int64_t nbytes) override {
+    return target_->Peek(nbytes);
+  }
+  bool supports_zero_copy() const override { return target_->supports_zero_copy(); }
+  Status Close() override { return target_->Close(); }
+  Status Abort() override { return target_->Abort(); }
+  Result<int64_t> Tell() const override { return target_->Tell(); }
+  bool closed() const override { return target_->closed(); }
+  Result<int64_t> Read(int64_t nbytes, void* out) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes, out);
+  }
+  Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes);
+  }
+  Status Seek(int64_t pos) { return target_->Seek(pos); }
+
+ private:
+  std::shared_ptr<io::BufferReader> target_;
+  double latency_s_;
+};
+
+static ReadOptions CreateReadOptions(bool use_threads, bool use_async) {
+  auto result = csv::ReadOptions::Defaults();
+  result.use_threads = use_threads;
+  result.legacy_blocking_reads = !use_async;
+  // Simulate larger files by using smaller block files so the impact of multiple
+  // blocks is seen but we don't have to spend the time waiting on the large I/O
+  result.block_size = (1 << 20) / 100;
+  return result;
+}
+
+static std::shared_ptr<SlowInputStream> CreateStreamReader(std::shared_ptr<Buffer> buffer,
+                                                           int64_t latency_ms) {
+  auto buffer_reader = std::make_shared<io::BufferReader>(buffer);
+  return std::make_shared<SlowInputStream>(buffer_reader, latency_ms);
+}
+
+static void BenchmarkReader(benchmark::State& state, bool use_threads, bool use_async) {
+  auto latency_ms = state.range(0);
+  auto num_rows = state.range(1);
+  auto num_files = state.range(2);
+  if (num_files > 5 && use_threads && !use_async) {
+    state.SkipWithError("Would deadlock");
+  }
+  auto input_buffer = *MakeSampleCsvBuffer(num_rows);
+  // Hard coding # of threads so we don't deadlock if there are too few cores
+  ASSIGN_OR_ABORT(auto thread_pool, internal::ThreadPool::Make(6));
+  io::AsyncContext async_context(thread_pool.get());
+  while (state.KeepRunning()) {
+    std::vector<Future<std::shared_ptr<Table>>> table_futures;
+    for (int i = 0; i < num_files; i++) {
+      auto stream_reader = CreateStreamReader(input_buffer, latency_ms);
+      auto table_reader = *csv::TableReader::Make(
+          default_memory_pool(), async_context, stream_reader,
+          CreateReadOptions(use_threads, use_async), csv::ParseOptions::Defaults(),
+          csv::ConvertOptions::Defaults());
+      if (use_async) {
+        table_futures.push_back(table_reader->ReadAsync());
+      } else {
+        ASSERT_OK_AND_ASSIGN(auto table_future,
+                             async_context.executor->Submit(
+                                 [table_reader] { return table_reader->Read(); }));
+        table_futures.push_back(table_future);
+      }
+    }
+    auto combined = All(table_futures);
+    ASSIGN_OR_ABORT(auto result, combined.result());
+    for (auto&& table : result) {
+      ABORT_NOT_OK(table);
+    }
+  }
+  state.SetItemsProcessed(state.iterations() * num_rows);

Review comment:
       Yes, good point.




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565929811



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      util::optional<Result<V>> next = Pump();
+      if (next.has_value()) {
+        return *next;

Review comment:
       Well, return value optimization (RVO) is allowed but apparently not mandatory. Also, in this case, the value being returned is `*next`, not `next.
   https://en.cppreference.com/w/cpp/language/copy_elision
   




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565176016



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}

Review comment:
       Ok. Regardless of the intent, though, it's better to test it :-)




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564750134



##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -837,20 +885,121 @@ class ThreadedTableReader : public BaseTableReader {
   }
 
  protected:
-  ThreadPool* thread_pool_;
+  Executor* thread_pool_;
+  Iterator<std::shared_ptr<Buffer>> buffer_iterator_;
+};
+
+class AsyncThreadedTableReader
+    : public BaseTableReader,
+      public std::enable_shared_from_this<AsyncThreadedTableReader> {
+ public:
+  using BaseTableReader::BaseTableReader;
+
+  AsyncThreadedTableReader(MemoryPool* pool, std::shared_ptr<io::InputStream> input,
+                           const ReadOptions& read_options,
+                           const ParseOptions& parse_options,
+                           const ConvertOptions& convert_options, Executor* thread_pool)
+      : BaseTableReader(pool, input, read_options, parse_options, convert_options),
+        thread_pool_(thread_pool) {}
+
+  ~AsyncThreadedTableReader() override {
+    if (task_group_) {
+      // In case of error, make sure all pending tasks are finished before
+      // we start destroying BaseTableReader members
+      ARROW_UNUSED(task_group_->Finish());
+    }
+  }
+
+  Status Init() override {
+    ARROW_ASSIGN_OR_RAISE(auto istream_it,
+                          io::MakeInputStreamIterator(input_, read_options_.block_size));
+
+    ARROW_ASSIGN_OR_RAISE(auto bg_it,
+                          MakeBackgroundIterator(std::move(istream_it), thread_pool_));
+
+    int32_t block_queue_size = thread_pool_->GetCapacity();
+    auto rh_it = AddReadahead(bg_it, block_queue_size);
+    buffer_generator_ = CSVBufferIterator::MakeAsync(std::move(rh_it));
+    return Status::OK();
+  }
+
+  Result<std::shared_ptr<Table>> Read() override { return ReadAsync().result(); }
+
+  Future<std::shared_ptr<Table>> ReadAsync() override {
+    task_group_ = internal::TaskGroup::MakeThreaded(thread_pool_);

Review comment:
       Ben was interested in this as well.  However, as it stands, there would be no clear benefit in doing so and it would require parallel async implementations of the column builder code (which currently rely on task group being present).  These parse & convert tasks do not block (except for maybe very briefly on shared collection access) and so using task group here isn't detrimental.
   
   If we want to remove TaskGroup for the purpose of simplifying down to a single "scheduler-like" interface then we could do that.  However, in that case, we should modify the existing serial & threaded table readers as well and I think it would make sense to do it as a separate story.




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



[GitHub] [arrow] ursabot commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-777374083


   ubuntu-20.04-x86_64: https://conbench.ursa.dev/compare/runs/a4551972-3bf9-4ba7-bef4-ec9ed683c09a...58851b68-ae2a-4993-a8b1-1dd094ee64b8/
   dgx-ubuntu-18.04-x86_64: https://conbench.ursa.dev/compare/runs/47769121-ed42-4698-89fe-3d851f2df615...911eec14-5e3e-40b9-b650-aaed785d3889/


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565972283



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(

Review comment:
       Removed.




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



[GitHub] [arrow] westonpace commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-771930077


   @ursabot please 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r572148239



##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -220,14 +242,36 @@ class ThreadedBlockReader : public BlockReader {
  public:
   using BlockReader::BlockReader;
 
-  Result<arrow::util::optional<CSVBlock>> Next() {
+  static Iterator<util::optional<CSVBlock>> MakeIterator(
+      Iterator<std::shared_ptr<Buffer>> buffer_iterator, std::unique_ptr<Chunker> chunker,
+      std::shared_ptr<Buffer> first_buffer) {
+    auto block_reader =
+        std::make_shared<ThreadedBlockReader>(std::move(chunker), first_buffer);
+    // Wrap shared pointer in callable
+    Transformer<std::shared_ptr<Buffer>, util::optional<CSVBlock>> block_reader_fn =
+        [block_reader](std::shared_ptr<Buffer> next) { return (*block_reader)(next); };
+    return MakeTransformedIterator(std::move(buffer_iterator), block_reader_fn);
+  }
+
+  static AsyncGenerator<util::optional<CSVBlock>> MakeAsyncIterator(
+      AsyncGenerator<std::shared_ptr<Buffer>> buffer_generator,
+      std::unique_ptr<Chunker> chunker, std::shared_ptr<Buffer> first_buffer) {
+    auto block_reader =
+        std::make_shared<ThreadedBlockReader>(std::move(chunker), first_buffer);
+    // Wrap shared pointer in callable
+    Transformer<std::shared_ptr<Buffer>, util::optional<CSVBlock>> block_reader_fn =
+        [block_reader](std::shared_ptr<Buffer> next) { return (*block_reader)(next); };
+    return TransformAsyncGenerator(std::move(buffer_generator), block_reader_fn);

Review comment:
       Nit: should make `MakeTransformedIterator` and `TransformAsyncGenerator` consistent, naming-wise.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r574208979



##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -220,14 +242,36 @@ class ThreadedBlockReader : public BlockReader {
  public:
   using BlockReader::BlockReader;
 
-  Result<arrow::util::optional<CSVBlock>> Next() {
+  static Iterator<util::optional<CSVBlock>> MakeIterator(
+      Iterator<std::shared_ptr<Buffer>> buffer_iterator, std::unique_ptr<Chunker> chunker,
+      std::shared_ptr<Buffer> first_buffer) {
+    auto block_reader =
+        std::make_shared<ThreadedBlockReader>(std::move(chunker), first_buffer);
+    // Wrap shared pointer in callable
+    Transformer<std::shared_ptr<Buffer>, util::optional<CSVBlock>> block_reader_fn =
+        [block_reader](std::shared_ptr<Buffer> next) { return (*block_reader)(next); };
+    return MakeTransformedIterator(std::move(buffer_iterator), block_reader_fn);
+  }
+
+  static AsyncGenerator<util::optional<CSVBlock>> MakeAsyncIterator(
+      AsyncGenerator<std::shared_ptr<Buffer>> buffer_generator,
+      std::unique_ptr<Chunker> chunker, std::shared_ptr<Buffer> first_buffer) {
+    auto block_reader =
+        std::make_shared<ThreadedBlockReader>(std::move(chunker), first_buffer);
+    // Wrap shared pointer in callable
+    Transformer<std::shared_ptr<Buffer>, util::optional<CSVBlock>> block_reader_fn =
+        [block_reader](std::shared_ptr<Buffer> next) { return (*block_reader)(next); };
+    return TransformAsyncGenerator(std::move(buffer_generator), block_reader_fn);

Review comment:
       I've changed all the functions that take in an AsyncGenerator and return an AsyncGenerator to  be MakeXYZ functions.  The only ones that aren't are the ones like Visit or Collect which take in AsyncGenerator and return Future.




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565172642



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {

Review comment:
       Yes. All those overheads tend to add up, though. Perhaps this is not overly concerning, but e.g. in the compute layer we're finding out that calling a kernel on relatively small batches has high fixed costs that we didn't envision, simply because of such overheads accumulating.
   
   (I'd also mention that `std::function` makes backtraces in debug mode rather annoying, but that's a much smaller concern)




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



[GitHub] [arrow] ursabot commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-777046036


   ubuntu-20.04-x86_64: https://conbench.ursa.dev/compare/runs/3aa6839d-e95c-4364-8e91-6edad18f0a72...84055e5f-88d9-44fe-9eb3-616c7502322f/
   dgx-ubuntu-18.04-x86_64: https://conbench.ursa.dev/compare/runs/cde6a063-c0f9-4364-8295-8742f5e44649...5c366831-0404-4910-aaf9-d2f88bf05afe/


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r566314807



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      util::optional<Result<V>> next = Pump();
+      if (next.has_value()) {
+        return *next;

Review comment:
       Ok, move added.  Thanks for bearing with me :)




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565177981



##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -177,14 +182,26 @@ class SerialBlockReader : public BlockReader {
  public:
   using BlockReader::BlockReader;
 
-  Result<arrow::util::optional<CSVBlock>> Next() {
+  static Iterator<util::optional<CSVBlock>> MakeIterator(
+      Iterator<std::shared_ptr<Buffer>> buffer_iterator, std::unique_ptr<Chunker> chunker,
+      std::shared_ptr<Buffer> first_buffer) {
+    auto block_reader =
+        std::make_shared<SerialBlockReader>(std::move(chunker), first_buffer);
+    // Wrap shared pointer in callable
+    Transformer<std::shared_ptr<Buffer>, util::optional<CSVBlock>> block_reader_fn =
+        [block_reader](std::shared_ptr<Buffer> buf) {
+          return (*block_reader)(std::move(buf));
+        };
+    return MakeTransformedIterator(std::move(buffer_iterator), block_reader_fn);
+  }
+
+  Result<TransformFlow<util::optional<CSVBlock>>> operator()(

Review comment:
       Ok, thanks for the explanation. Can you add that somewhere as a comment?




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r566551201



##########
File path: cpp/src/arrow/util/iterator.cc
##########
@@ -119,14 +123,30 @@ class ReadaheadQueue::Impl : public std::enable_shared_from_this<ReadaheadQueue:
   void DoWork() {
     std::unique_lock<std::mutex> lock(mutex_);
     while (!please_shutdown_) {
-      while (static_cast<int64_t>(done_.size()) < max_readahead_ && todo_.size() > 0) {
+      while (todo_.size() > 0 &&
+             ((max_readahead_ <= 0) ||
+              (static_cast<int64_t>(done_.size()) < max_readahead_))) {
         auto promise = std::move(todo_.front());
         todo_.pop_front();
         lock.unlock();
-        promise->Call();
+        if (promise->Call()) {
+          // If the call finished then we should purge the remaining TODO items, marking
+          // them finished
+          lock.lock();
+          std::deque<std::unique_ptr<ReadaheadPromise>> to_clear(std::move(todo_));
+          // While the async iterator doesn't use todo_ anymore after it hits a finish the
+          // sync iterator might still due to timing so leave it valid
+          todo_.clear();
+          lock.unlock();
+          for (auto&& promise : to_clear) {
+            promise->End();
+          }
+        }
         lock.lock();
-        done_.push_back(std::move(promise));
-        work_done_.notify_one();
+        if (max_readahead_ > 0) {
+          done_.push_back(std::move(promise));
+          work_done_.notify_one();
+        }

Review comment:
       Ok.  I've tracked down the oddness here and it actually unwound quite a bit of complexity.  It turns out that readahead is something of an inherently asynchronous concept.  The ReadaheadQueue was actually an asynchronous generator pipeline in disguise.
   
   ReadaheadQueueImpl is actually part executor and part AddReadahead.
   
   Part executor: ReadaheadQueueImpl is a thread pool of size 1.  It has a worker thread (the background thread) and a job queue (ReadaheadQueueImpl::todo_).  It accepted jobs on the job queue and ran them.  The types were even named "promises".
   
   Part readahead: It was also part readahead.  The readahead queue was ReadaheadQueueImpl::done_ and all of the logic that was in the pumping.
   
   So I got rid of it and replaced it with a thread pool of size 1 (which has an unlimited size job queue), BackgroundGenerator (which now takes in an executor and an iterator and creates an AsyncGenerator), AddReadahead (which creates readahead queue, this one is limited in size), and finally GeneratorIterator which brings us back to a synchronous iterator by waiting each result.
   
   The tests all pass, except the logic changed a little bit on the tracing test because the readahead now doesn't pump until the first read but we can change that if we want to so it is more like the threaded reader and less like the serial reader.
   
   I've made the change on a separate branch from this one so that change can be reviewed independently (https://github.com/westonpace/arrow/pull/3).




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564768704



##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NTASKS = 100;
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+
+  std::vector<Future<std::shared_ptr<Table>>> task_futures(NTASKS);
+  for (int i = 0; i < NTASKS; i++) {
+    auto input = std::make_shared<io::BufferReader>(table_buffer);
+    ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+    task_futures[i] = reader->ReadAsync();
+  }
+  auto combined_future = All(task_futures);
+  combined_future.Wait();
+
+  ASSERT_OK_AND_ASSIGN(std::vector<Result<std::shared_ptr<Table>>> results,
+                       combined_future.result());
+  for (auto&& result : results) {
+    ASSERT_OK_AND_ASSIGN(auto table, result);
+    ASSERT_EQ(NROWS, table->num_rows());
+  }
+}
+
+void TestNestedParallelism(
+    std::shared_ptr<internal::ThreadPool> thread_pool,
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+  auto input = std::make_shared<io::BufferReader>(table_buffer);
+  ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+
+  Future<std::shared_ptr<Table>> table_future;
+
+  auto read_task = [&reader, &table_future]() mutable {
+    table_future = reader->ReadAsync();
+    return Status::OK();
+  };
+  ASSERT_OK_AND_ASSIGN(auto future, thread_pool->Submit(read_task));
+  ASSERT_TRUE(future.Wait(1));
+
+  if (future.is_finished()) {
+    ASSERT_TRUE(table_future.Wait(1));
+    if (table_future.is_finished()) {
+      ASSERT_OK_AND_ASSIGN(auto table, table_future.result());
+      ASSERT_EQ(table->num_rows(), NROWS);
+    }
+  }
+}  // namespace csv
+
+TEST(SerialReaderTests, Stress) {
+  auto task_factory = [](std::shared_ptr<io::InputStream> input_stream) {
+    return TableReader::Make(default_memory_pool(), io::AsyncContext(), input_stream,
+                             ReadOptions::Defaults(), ParseOptions::Defaults(),
+                             ConvertOptions::Defaults());
+  };
+  StressTableReader(task_factory);
+}
+
+TEST(SerialReaderTests, NestedParallelism) {
+  ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1));

Review comment:
       Well, this is the serial reader, so it doesn't make use of a thread pool.  However, I could pass one in the `io::AsyncContext` anyway for good measure.  It's not harmless in the async or the threaded (commented out) cases because the thread pool is passed in via the `io::AsyncContext` and is reused by the readers.  There is a reason the threaded case is commented out (it deadlocks).




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564827609



##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,9 +832,203 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(3, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(Status::IOError("XYZ"), results[1].status());
+    ASSERT_EQ(3, *results[2]);
+  }));
+
+  f1.MarkFinished(1);
+  f2.MarkFinished(Status::IOError("XYZ"));
+  f3.MarkFinished(3);
+
+  AssertFinished(combined);
+}
+
+TEST(FutureLoopTest, Sync) {
+  struct {
+    int i = 0;
+    Future<int> Get() { return Future<int>::MakeFinished(i++); }
+  } IntSource;
+
+  bool do_fail = false;
+  std::vector<int> ints;
+  auto loop_body = [&] {
+    return IntSource.Get().Then([&](int i) -> Result<ControlFlow<int>> {
+      if (do_fail && i == 3) {
+        return Status::IOError("xxx");
+      }
+
+      if (i == 5) {
+        int sum = 0;
+        for (int i : ints) sum += i;
+        return Break(sum);
+      }
+
+      ints.push_back(i);
+      return Continue();
+    });
+  };
+
+  {
+    auto sum_fut = Loop(loop_body);
+    AssertSuccessful(sum_fut);
+
+    ASSERT_OK_AND_ASSIGN(auto sum, sum_fut.result());
+    ASSERT_EQ(sum, 0 + 1 + 2 + 3 + 4);
+  }
+
+  {
+    do_fail = true;
+    IntSource.i = 0;
+    auto sum_fut = Loop(loop_body);
+    AssertFailed(sum_fut);
+    ASSERT_RAISES(IOError, sum_fut.result());
+  }
+}
+
+TEST(FutureLoopTest, EmptyBreakValue) {
+  Future<> none_fut =
+      Loop([&] { return Future<>::MakeFinished().Then([&](...) { return Break(); }); });
+  AssertSuccessful(none_fut);
+}
+
+TEST(FutureLoopTest, MoveOnlyBreakValue) {
+  Future<MoveOnlyDataType> one_fut = Loop([&] {
+    return Future<int>::MakeFinished(1).Then(
+        [&](int i) { return Break(MoveOnlyDataType(i)); });
+  });
+  AssertSuccessful(one_fut);
+  ASSERT_OK_AND_ASSIGN(auto one, std::move(one_fut).result());
+  ASSERT_EQ(one, 1);
+}
+
+TEST(FutureLoopTest, StackOverflow) {
+  // Looping over futures is normally a rather recursive task.  If the futures complete
+  // synchronously (because they are already finished) it could lead to a stack overflow
+  // if care is not taken.
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 1000000) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());
+    }
+    return Future<ControlFlow<int>>::MakeFinished(Break(-1));
+  };
+  auto loop_fut = Loop(loop_body);
+  ASSERT_TRUE(loop_fut.Wait(0.1));
+}
+
+TEST(FutureLoopTest, AllowsBreakFutToBeDiscarded) {
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 10) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());
+    }
+    return Future<ControlFlow<int>>::MakeFinished(Break(-1));
+  };
+  auto loop_fut = Loop(loop_body).Then([](...) { return Status::OK(); });
+  ASSERT_TRUE(loop_fut.Wait(0.1));
+}
+
+TEST(FutureLoopTest, EmptyLoop) {
+  auto loop_body = []() -> Future<ControlFlow<int>> {
+    return Future<ControlFlow<int>>::MakeFinished(Break(0));
+  };
+  auto loop_fut = Loop(loop_body);
+  ASSERT_TRUE(loop_fut.Wait(0.1));
+  if (loop_fut.is_finished()) {

Review comment:
       If `loop_fut` has a bug and never marks complete then `ASSERT_TRUE(loop_fut.Wait(0.1))` will fail but not abort.  The call to `result` will then deadlock which is an undesirable thing for a test case to do.  Can replace with `ASSERT_FINISHES_AND_ASSIGN`




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565983559



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.
+template <typename T>
+class BackgroundIterator {
+  using PromiseType = typename detail::BackgroundIteratorPromise<T>;
+
+ public:
+  explicit BackgroundIterator(Iterator<T> it, internal::Executor* executor)
+      : it_(new Iterator<T>(std::move(it))),
+        queue_(new detail::ReadaheadQueue(0)),
+        executor_(executor),
+        done_() {}
+
+  ~BackgroundIterator() {
+    if (queue_) {
+      // Make sure the queue doesn't call any promises after this object
+      // is destroyed.
+      queue_->EnsureShutdownOrDie();
+    }
+  }
+
+  ARROW_DEFAULT_MOVE_AND_ASSIGN(BackgroundIterator);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(BackgroundIterator);
+
+  Future<T> operator()() {
+    if (done_) {
+      return Future<T>::MakeFinished(IterationTraits<T>::End());
+    }
+    auto promise = std::unique_ptr<PromiseType>(new PromiseType{it_.get()});
+    auto result = Future<T>(promise->out_);
+    // TODO: Need a futuristic version of ARROW_RETURN_NOT_OK
+    auto append_status = queue_->Append(
+        static_cast<std::unique_ptr<detail::ReadaheadPromise>>(std::move(promise)));
+    if (!append_status.ok()) {
+      return Future<T>::MakeFinished(append_status);
+    }
+
+    result.AddCallback([this](const Result<T>& result) {
+      if (!result.ok() || result.ValueUnsafe() == IterationTraits<T>::End()) {
+        done_ = true;
+      }
+    });
+
+    return executor_->Transfer(result);

Review comment:
       But is it the background generator's job to call `Transfer`? Is the caller simply meant to pass the "executor of the current thread"?
   
   It seems at some point we should add the notion of a per-thread currently running executor. What do you 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565979145



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue

Review comment:
       Fair enough :-)




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



[GitHub] [arrow] westonpace edited a comment on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace edited a comment on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-777175290


   @pitrou It now runs on the CPU thread pool.  The I/O thread pool is sent down all the way into the table reader but is not yet used (I added ARROW-11590 for this).  When I tried to run it on the I/O thread pool it failed because there were multiple threads running the iterator.  The current behavior (dedicated single thread pool) matches the old behavior though so this isn't a regression.
   
   I think I've addressed the major concerns and it is ready for review again.


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r555260697



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}

Review comment:
       Done.

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      util::optional<Result<V>> next = Pump();
+      if (next.has_value()) {
+        return *next;
+      }
+      ARROW_ASSIGN_OR_RAISE(last_value_, it_.Next());
+    }
+    return IterationTraits<V>::End();
+  }
+
+ private:
+  Iterator<T> it_;
+  Transformer<T, V> transformer_;
+  util::optional<T> last_value_;
+  bool finished_;

Review comment:
       Done

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;

Review comment:
       Done




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565969521



##########
File path: cpp/src/arrow/util/thread_pool.h
##########
@@ -87,6 +87,21 @@ class ARROW_EXPORT Executor {
     return SpawnReal(hints, std::forward<Function>(func));
   }
 
+  template <typename T>
+  Future<T> Transfer(Future<T> future) {
+    auto transferred = Future<T>::Make();
+    future.AddCallback([this, transferred](const Result<T>& result) mutable {
+      Result<T> result_copy(result);
+      auto spawn_status = Spawn([transferred, result_copy]() mutable {

Review comment:
       This copy is unavoidable.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565957034



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +615,74 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>

Review comment:
       It can.  I converted it to be a typedef instead of a separate struct.

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +615,74 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>

Review comment:
       It can.  I converted it to be a type alias instead of a separate struct.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565971541



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}

Review comment:
       Done.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565983607



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.
+template <typename T>
+class BackgroundIterator {
+  using PromiseType = typename detail::BackgroundIteratorPromise<T>;
+
+ public:
+  explicit BackgroundIterator(Iterator<T> it, internal::Executor* executor)
+      : it_(new Iterator<T>(std::move(it))),
+        queue_(new detail::ReadaheadQueue(0)),
+        executor_(executor),
+        done_() {}
+
+  ~BackgroundIterator() {
+    if (queue_) {
+      // Make sure the queue doesn't call any promises after this object
+      // is destroyed.
+      queue_->EnsureShutdownOrDie();
+    }
+  }
+
+  ARROW_DEFAULT_MOVE_AND_ASSIGN(BackgroundIterator);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(BackgroundIterator);
+
+  Future<T> operator()() {
+    if (done_) {
+      return Future<T>::MakeFinished(IterationTraits<T>::End());
+    }
+    auto promise = std::unique_ptr<PromiseType>(new PromiseType{it_.get()});
+    auto result = Future<T>(promise->out_);

Review comment:
       Renamed to `future`.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565957477



##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,9 +832,203 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {

Review comment:
       Done.

##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,9 +832,203 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {

Review comment:
       Done.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564778430



##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NTASKS = 100;
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+
+  std::vector<Future<std::shared_ptr<Table>>> task_futures(NTASKS);
+  for (int i = 0; i < NTASKS; i++) {
+    auto input = std::make_shared<io::BufferReader>(table_buffer);
+    ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+    task_futures[i] = reader->ReadAsync();
+  }
+  auto combined_future = All(task_futures);
+  combined_future.Wait();
+
+  ASSERT_OK_AND_ASSIGN(std::vector<Result<std::shared_ptr<Table>>> results,
+                       combined_future.result());
+  for (auto&& result : results) {
+    ASSERT_OK_AND_ASSIGN(auto table, result);
+    ASSERT_EQ(NROWS, table->num_rows());
+  }
+}
+
+void TestNestedParallelism(
+    std::shared_ptr<internal::ThreadPool> thread_pool,
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+  auto input = std::make_shared<io::BufferReader>(table_buffer);
+  ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+
+  Future<std::shared_ptr<Table>> table_future;
+
+  auto read_task = [&reader, &table_future]() mutable {
+    table_future = reader->ReadAsync();
+    return Status::OK();
+  };
+  ASSERT_OK_AND_ASSIGN(auto future, thread_pool->Submit(read_task));
+  ASSERT_TRUE(future.Wait(1));
+
+  if (future.is_finished()) {
+    ASSERT_TRUE(table_future.Wait(1));
+    if (table_future.is_finished()) {
+      ASSERT_OK_AND_ASSIGN(auto table, table_future.result());
+      ASSERT_EQ(table->num_rows(), NROWS);
+    }
+  }
+}  // namespace csv
+
+TEST(SerialReaderTests, Stress) {
+  auto task_factory = [](std::shared_ptr<io::InputStream> input_stream) {
+    return TableReader::Make(default_memory_pool(), io::AsyncContext(), input_stream,
+                             ReadOptions::Defaults(), ParseOptions::Defaults(),
+                             ConvertOptions::Defaults());
+  };
+  StressTableReader(task_factory);
+}
+
+TEST(SerialReaderTests, NestedParallelism) {
+  ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1));
+  auto task_factory = [](std::shared_ptr<io::InputStream> input_stream) {
+    return TableReader::Make(default_memory_pool(), io::AsyncContext(), input_stream,
+                             ReadOptions::Defaults(), ParseOptions::Defaults(),
+                             ConvertOptions::Defaults());
+  };
+  TestNestedParallelism(thread_pool, task_factory);
+}
+
+TEST(ThreadedReaderTests, Stress) {
+  ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1));
+  auto task_factory = [&thread_pool](std::shared_ptr<io::InputStream> input_stream)
+      -> Result<std::shared_ptr<TableReader>> {
+    ReadOptions read_options = ReadOptions::Defaults();
+    read_options.use_threads = true;
+    read_options.legacy_blocking_reads = true;
+    auto table_reader = TableReader::Make(
+        default_memory_pool(), io::AsyncContext(thread_pool.get()), input_stream,

Review comment:
       Hmm...if that is the purpose of `io::AsyncContext` then I have misused it.  I don't think that *should* be the purpose though.  It's possible I am wrong.  Although looking at the existing implementation I think I am indeed in misalignment...
   ```
   // Default ReadAsync() implementation: simply issue the read on the context's executor
   Future<std::shared_ptr<Buffer>> RandomAccessFile::ReadAsync(const AsyncContext& ctx, ...) {
     return DeferNotOk(ctx.executor->Submit(...);
   }
   ```
   ...this logic doesn't work I don't think.  Well, it puts the burden on the user to transfer the continuation to the CPU context.  As it stands, if the user added a continuation, that continuation would run on the IO context.
   
   When I see something like...
   
   ```
   Future SomeIOOperation(AsyncContext ctx)
   ```
   
   In my mind `ctx` is "This is the context on which work should resume when the I/O operation completes".  Indeed, it is the CPU context which has to be async most of the time.  There is typically not much harm in having a synchronous I/O context.
   
   However, it sounds like the existing definition is "This is the context on which the I/O operation should run".  However, I think that should be owned by the FS.  Maybe AWS has a thread pool of its own that manages its operations.  Maybe a memory wrapping filesystem uses a single synchronous thread.  Maybe the FS is using libuv which has its own background thread pool.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565938785



##########
File path: cpp/src/arrow/csv/test_common.cc
##########
@@ -61,5 +61,47 @@ void MakeColumnParser(std::vector<std::string> items, std::shared_ptr<BlockParse
   ASSERT_EQ((*out)->num_rows(), items.size());
 }
 
+const std::vector<std::string> int64_rows = {"123", "4", "-317005557", "", "N/A", "0"};
+const std::vector<std::string> float_rows = {"0", "123.456", "-3170.55766", "", "N/A"};
+const std::vector<std::string> decimal128_rows = {"0", "123.456", "-3170.55766",
+                                                  "",  "N/A",     "1233456789.123456789"};
+const std::vector<std::string> iso8601_rows = {"1917-10-17", "2018-09-13",
+                                               "1941-06-22 04:00", "1945-05-09 09:45:38"};
+const std::vector<std::string> strptime_rows = {"10/17/1917", "9/13/2018", "9/5/1945"};
+
+static void WriteHeader(std::ostream& writer) {
+  writer << "Int64,Float,Decimal128,ISO8601,Strptime" << std::endl;
+}
+
+static std::string GetCell(std::vector<std::string> base_rows, size_t row_index) {

Review comment:
       I changed to a const-ref




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565935224



##########
File path: cpp/src/arrow/csv/reader_benchmark.cc
##########
@@ -0,0 +1,139 @@
+// 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 <string>
+
+#include "arrow/buffer.h"
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+class SlowInputStream : public io::InputStream {

Review comment:
       This benchmark has been removed from the PR.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565983607



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.
+template <typename T>
+class BackgroundIterator {
+  using PromiseType = typename detail::BackgroundIteratorPromise<T>;
+
+ public:
+  explicit BackgroundIterator(Iterator<T> it, internal::Executor* executor)
+      : it_(new Iterator<T>(std::move(it))),
+        queue_(new detail::ReadaheadQueue(0)),
+        executor_(executor),
+        done_() {}
+
+  ~BackgroundIterator() {
+    if (queue_) {
+      // Make sure the queue doesn't call any promises after this object
+      // is destroyed.
+      queue_->EnsureShutdownOrDie();
+    }
+  }
+
+  ARROW_DEFAULT_MOVE_AND_ASSIGN(BackgroundIterator);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(BackgroundIterator);
+
+  Future<T> operator()() {
+    if (done_) {
+      return Future<T>::MakeFinished(IterationTraits<T>::End());
+    }
+    auto promise = std::unique_ptr<PromiseType>(new PromiseType{it_.get()});
+    auto result = Future<T>(promise->out_);

Review comment:
       Renamed to `future`.  I'll have to break my habit of always calling the "thing I'm going to return" `result`. :)




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



[GitHub] [arrow] nealrichardson commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
nealrichardson commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r570465569



##########
File path: cpp/src/arrow/csv/options.h
##########
@@ -119,6 +119,13 @@ struct ARROW_EXPORT ReadOptions {
 
   /// Whether to use the global CPU thread pool
   bool use_threads = true;
+  /// TODO: This will come out before this gets merged in.  There is not much point in

Review comment:
       Ready to remove this?




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565970448



##########
File path: cpp/src/arrow/util/thread_pool.h
##########
@@ -87,6 +87,21 @@ class ARROW_EXPORT Executor {
     return SpawnReal(hints, std::forward<Function>(func));
   }
 
+  template <typename T>
+  Future<T> Transfer(Future<T> future) {

Review comment:
       I added a comment.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564834775



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +187,122 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_ = false;
+  bool ready_for_next_ = false;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<Result<TransformFlow<V>>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  // Calls the transform function on the current value.  Can return in several ways
+  // * If the next value is requested (e.g. skip) it will return an empty optional
+  // * If an invalid status is encountered that will be returned
+  // * If finished it will return IterationTraits<V>::End()
+  // * If a value is returned by the transformer that will be returned
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      ARROW_ASSIGN_OR_RAISE(util::optional<V> next, Pump());
+      if (next.has_value()) {
+        return *next;
+      }
+      ARROW_ASSIGN_OR_RAISE(last_value_, it_.Next());
+    }
+    return IterationTraits<V>::End();
+  }
+
+ private:
+  Iterator<T> it_;
+  Transformer<T, V> transformer_;
+  util::optional<T> last_value_;
+  bool finished_ = false;
+};
+
+/// \brief Transforms an iterator according to a transformer, returning a new Iterator.
+///
+/// The transformer will be called on each element of the source iterator and for each
+/// call it can yield a value, skip, or finish the iteration.  When yielding a value the
+/// transformer can choose to consume the source item (the default, ready_for_next = true)
+/// or to keep it and it will be called again on the same value.

Review comment:
       A decompression iterator.  If the iterator consumes 1MB blocks and outputs 1MB blocks it will likely need to output N blocks for each input block.  Or a flatMap operator which takes in lists and outputs elements.  For example, in a dataset situation you could conceivably encounter `AsyncIterator<Dataset> -> AsyncIterator<Directory> -> AsyncIterator<File> -> AsyncIterator<Segment>`  Each element in the chain would need to output multiple elements for each input.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565979874



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {

Review comment:
       I changed the tests to transform from `TestInt` to a newly created `TestStr`




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564847646



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}

Review comment:
       For the `Transformer` no.  A `Transformer` cannot be called in parallel (this may be a limitation we have to revisit).   For an `AsyncGenerator` in general, perhaps, it is undefined.  Imagine your source is a CSV file stored on AWS and so you are issuing 8 reads in parallel.  It's entirely feasible that the 3rd read fails while the other reads succeed.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r576334652



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -581,4 +646,82 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+struct Continue {
+  template <typename T>
+  operator util::optional<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+util::optional<T> Break(T break_value = {}) {
+  return util::optional<T>{std::move(break_value)};
+}
+
+template <typename T = detail::Empty>
+using ControlFlow = util::optional<T>;
+
+/// \brief Loop through an asynchronous sequence
+///
+/// \param[in] iterate A generator of Future<ControlFlow<BreakValue>>. On completion of
+/// each yielded future the resulting ControlFlow will be examined. A Break will terminate
+/// the loop, while a Continue will re-invoke `iterate`. \return A future which will
+/// complete when a Future returned by iterate completes with a Break
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::value_type>
+Future<BreakValueType> Loop(Iterate iterate) {
+  auto break_fut = Future<BreakValueType>::Make();
+
+  struct Callback {
+    bool CheckForTermination(const Result<Control>& control_res) {
+      if (!control_res.ok()) {
+        break_fut.MarkFinished(control_res.status());
+        return true;
+      }
+      if (control_res->has_value()) {
+        break_fut.MarkFinished(*std::move(*control_res));
+        return true;
+      }
+      return false;
+    }
+
+    void operator()(const Result<Control>& maybe_control) && {
+      if (CheckForTermination(maybe_control)) return;
+
+      auto control_fut = iterate();
+      while (true) {
+        if (control_fut.is_finished()) {
+          // There's no need to AddCallback on a finished future; we can
+          // CheckForTermination now. This also avoids recursion and potential stack
+          // overflow.
+          if (CheckForTermination(control_fut.result())) return;
+
+          control_fut = iterate();
+        } else {
+          std::function<Callback()> callback_factory = [this]() { return *this; };
+          if (control_fut.TryAddCallback(callback_factory)) {
+            break;
+          }
+          // Else we tried to add a callback but someone had stolen in and marked the
+          // future finished so we can just resume iteration
+        }

Review comment:
       I agree that is simpler, I merged it in.




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565176206



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}
+
+TEST(TestAsyncIteratorTransform, SkipSome) {
+  auto original = AsyncVectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = TransformAsyncGenerator(std::move(original), filter);
+  AssertAsyncGeneratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestAsyncUtil, ReadaheadFailed) {
+  auto source = []() -> Future<TestInt> {
+    return Future<TestInt>::MakeFinished(Status::Invalid("X"));
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  auto next = readahead();
+  ASSERT_EQ(Status::Invalid("X"), next.status());

Review comment:
       Same as above: regardless of the intent, it's better to test it :-)




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565176787



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}
+
+TEST(TestAsyncIteratorTransform, SkipSome) {
+  auto original = AsyncVectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = TransformAsyncGenerator(std::move(original), filter);
+  AssertAsyncGeneratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestAsyncUtil, ReadaheadFailed) {
+  auto source = []() -> Future<TestInt> {
+    return Future<TestInt>::MakeFinished(Status::Invalid("X"));
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  auto next = readahead();
+  ASSERT_EQ(Status::Invalid("X"), next.status());
+}
+
+TEST(TestAsyncUtil, Readahead) {
+  int num_delivered = 0;
+  auto source = [&num_delivered]() {
+    if (num_delivered < 5) {
+      return Future<TestInt>::MakeFinished(num_delivered++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  // Should not pump until first item requested

Review comment:
       Reading from the source file before `Read()` is called is by design, so that the reading can start in the background without the caller having to wait.
   
   You're right that for an async reader it's much less useful, though.
   




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



[GitHub] [arrow] bkietz commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r555975697



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +611,73 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>
+Future<BreakValueType> Loop(Iterate iterate) {

Review comment:
       NVM; type deduction isn't clever enough to infer return type like this




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564809390



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {

Review comment:
       What sort of optimization are you concerned about?  Is it the std::function overhead?  Because I agree, I don't think the compiler is going to optimize that away.  This visitor is probably going to be called on a per-block basis and shouldn't really be used on a per-element basis.  The overhead of calling the function is likely to be on par with the overhead already existing waiting for a future to complete, adding the callback, executing the callback, etc. 




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564815408



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -556,6 +559,38 @@ inline bool WaitForAll(const std::vector<Future<T>*>& futures,
   return waiter->Wait(seconds);
 }
 
+/// \brief Create a Future which completes when all of `futures` complete.
+///
+/// The future's result is a vector of the results of `futures`.
+/// Note that this future will never be marked "failed"; failed results
+/// will be stored in the result vector alongside successful results.
+template <typename T>
+Future<std::vector<Result<T>>> All(std::vector<Future<T>> futures) {
+  struct State {
+    explicit State(std::vector<Future<T>> f)
+        : futures(std::move(f)), n_remaining(futures.size()) {}
+
+    std::vector<Future<T>> futures;
+    std::atomic<size_t> n_remaining;
+  };

Review comment:
       All versions of `FutureWaiter` wait (put the thread in a non-runnable state waiting on the CV) if I read it correctly.  This version adds the "after-wait" as a continuation that runs as a callback  when the last future being waited on completes.
   ![Untitled presentation](https://user-images.githubusercontent.com/1696093/105902075-3e648000-5fc2-11eb-80c0-d59e65d78b23.png)
   




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565939041



##########
File path: cpp/src/arrow/csv/test_common.h
##########
@@ -22,6 +22,7 @@
 #include <vector>
 
 #include "arrow/csv/parser.h"
+#include "arrow/io/memory.h"

Review comment:
       I removed the include




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565935965



##########
File path: cpp/src/arrow/csv/test_common.cc
##########
@@ -61,5 +61,47 @@ void MakeColumnParser(std::vector<std::string> items, std::shared_ptr<BlockParse
   ASSERT_EQ((*out)->num_rows(), items.size());
 }
 
+const std::vector<std::string> int64_rows = {"123", "4", "-317005557", "", "N/A", "0"};
+const std::vector<std::string> float_rows = {"0", "123.456", "-3170.55766", "", "N/A"};
+const std::vector<std::string> decimal128_rows = {"0", "123.456", "-3170.55766",
+                                                  "",  "N/A",     "1233456789.123456789"};
+const std::vector<std::string> iso8601_rows = {"1917-10-17", "2018-09-13",
+                                               "1941-06-22 04:00", "1945-05-09 09:45:38"};
+const std::vector<std::string> strptime_rows = {"10/17/1917", "9/13/2018", "9/5/1945"};

Review comment:
       I think that's ok.  It wasn't particularly relevant to what I was testing.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565977338



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);

Review comment:
       I'm not entirely certain I understand the comment.  Are you saying I should check to see if `back_of_queue` is finished and, if so, mark the iterator finished immediately instead of via a callback?  It's not likely to add much overhead since `AddCallback` will execute synchronously in that case anyways.  Also, there is no stack overflow concern here because we are, at most, adding one callback per call.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565964386



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);

Review comment:
       Yes, though the test was passing so I'm not sure if some kind of implicit conversion was happening.  I have made it correct and more explicit with the new ASSERT_FINISHES...

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous

Review comment:
       Done




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564754374



##########
File path: cpp/src/arrow/util/iterator.cc
##########
@@ -119,14 +123,30 @@ class ReadaheadQueue::Impl : public std::enable_shared_from_this<ReadaheadQueue:
   void DoWork() {
     std::unique_lock<std::mutex> lock(mutex_);
     while (!please_shutdown_) {
-      while (static_cast<int64_t>(done_.size()) < max_readahead_ && todo_.size() > 0) {
+      while (todo_.size() > 0 &&
+             ((max_readahead_ <= 0) ||
+              (static_cast<int64_t>(done_.size()) < max_readahead_))) {
         auto promise = std::move(todo_.front());
         todo_.pop_front();
         lock.unlock();
-        promise->Call();
+        if (promise->Call()) {
+          // If the call finished then we should purge the remaining TODO items, marking
+          // them finished
+          lock.lock();
+          std::deque<std::unique_ptr<ReadaheadPromise>> to_clear(std::move(todo_));
+          // While the async iterator doesn't use todo_ anymore after it hits a finish the
+          // sync iterator might still due to timing so leave it valid
+          todo_.clear();
+          lock.unlock();
+          for (auto&& promise : to_clear) {
+            promise->End();
+          }
+        }
         lock.lock();
-        done_.push_back(std::move(promise));
-        work_done_.notify_one();
+        if (max_readahead_ > 0) {
+          done_.push_back(std::move(promise));
+          work_done_.notify_one();
+        }

Review comment:
       I think we will want a general purpose producer/consumer queue with both synchronous and asynchronous APIs.  The AddReadahead is bounded by the "done" pool is in AddReadahead (and not in the readahead queue).  I agree it needs some simplification but I found the old method equally confusing.  Both approaches should be able to share a single multi-producer / multi-consumer "queue" API (maybe single-consumer / single-producer is sufficient if there are performance benefits to such a queue).
   
   I will take a pass at simplifying this.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564867541



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -16,6 +16,7 @@
 // under the License.
 
 #include "arrow/util/iterator.h"
+#include "arrow/util/async_iterator.h"

Review comment:
       Clang ordered them this way :man_shrugging: .  I just tried and it was reproducible.  Fortunately, if I group these includes with the `gtest_util.h` include then it does the right thing.




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



[GitHub] [arrow] ursabot commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-777168374


   ubuntu-20.04-x86_64: https://conbench.ursa.dev/compare/runs/a4551972-3bf9-4ba7-bef4-ec9ed683c09a...bf76bd78-a3e4-4cee-952a-cd5ee332e4e6/
   dgx-ubuntu-18.04-x86_64: https://conbench.ursa.dev/compare/runs/47769121-ed42-4698-89fe-3d851f2df615...a7a827b4-b52d-460e-851a-d7a32f971ad1/


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565981039



##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -177,14 +182,26 @@ class SerialBlockReader : public BlockReader {
  public:
   using BlockReader::BlockReader;
 
-  Result<arrow::util::optional<CSVBlock>> Next() {
+  static Iterator<util::optional<CSVBlock>> MakeIterator(
+      Iterator<std::shared_ptr<Buffer>> buffer_iterator, std::unique_ptr<Chunker> chunker,
+      std::shared_ptr<Buffer> first_buffer) {
+    auto block_reader =
+        std::make_shared<SerialBlockReader>(std::move(chunker), first_buffer);
+    // Wrap shared pointer in callable
+    Transformer<std::shared_ptr<Buffer>, util::optional<CSVBlock>> block_reader_fn =
+        [block_reader](std::shared_ptr<Buffer> buf) {
+          return (*block_reader)(std::move(buf));
+        };
+    return MakeTransformedIterator(std::move(buffer_iterator), block_reader_fn);
+  }
+
+  Result<TransformFlow<util::optional<CSVBlock>>> operator()(

Review comment:
       I added a comment on BlockReader.




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564534403



##########
File path: cpp/src/arrow/csv/reader_benchmark.cc
##########
@@ -0,0 +1,139 @@
+// 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 <string>
+
+#include "arrow/buffer.h"
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+class SlowInputStream : public io::InputStream {

Review comment:
       Hmm... why don't you use `SlowInputStream` from `arrow/io/slow.h`?

##########
File path: cpp/src/arrow/csv/reader_benchmark.cc
##########
@@ -0,0 +1,139 @@
+// 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 <string>
+
+#include "arrow/buffer.h"
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+class SlowInputStream : public io::InputStream {
+ public:
+  explicit SlowInputStream(std::shared_ptr<io::BufferReader> target, int64_t latency_ms)
+      : target_(std::move(target)) {
+    latency_s_ = static_cast<double>(latency_ms) / 1000.0;
+  }
+  virtual ~SlowInputStream() {}
+
+  Result<util::string_view> Peek(int64_t nbytes) override {
+    return target_->Peek(nbytes);
+  }
+  bool supports_zero_copy() const override { return target_->supports_zero_copy(); }
+  Status Close() override { return target_->Close(); }
+  Status Abort() override { return target_->Abort(); }
+  Result<int64_t> Tell() const override { return target_->Tell(); }
+  bool closed() const override { return target_->closed(); }
+  Result<int64_t> Read(int64_t nbytes, void* out) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes, out);
+  }
+  Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes);
+  }
+  Status Seek(int64_t pos) { return target_->Seek(pos); }
+
+ private:
+  std::shared_ptr<io::BufferReader> target_;
+  double latency_s_;
+};
+
+static ReadOptions CreateReadOptions(bool use_threads, bool use_async) {
+  auto result = csv::ReadOptions::Defaults();
+  result.use_threads = use_threads;
+  result.legacy_blocking_reads = !use_async;
+  // Simulate larger files by using smaller block files so the impact of multiple
+  // blocks is seen but we don't have to spend the time waiting on the large I/O

Review comment:
       I'm not sure this is exactly the same. Using a very small block size (I assume the below is 10kB?) may emphasize fixed costs (managing vectors of shared_ptrs etc.) rather than actual reading/parsing costs.
   

##########
File path: cpp/src/arrow/csv/reader_benchmark.cc
##########
@@ -0,0 +1,139 @@
+// 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 <string>
+
+#include "arrow/buffer.h"
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+class SlowInputStream : public io::InputStream {
+ public:
+  explicit SlowInputStream(std::shared_ptr<io::BufferReader> target, int64_t latency_ms)
+      : target_(std::move(target)) {
+    latency_s_ = static_cast<double>(latency_ms) / 1000.0;
+  }
+  virtual ~SlowInputStream() {}
+
+  Result<util::string_view> Peek(int64_t nbytes) override {
+    return target_->Peek(nbytes);
+  }
+  bool supports_zero_copy() const override { return target_->supports_zero_copy(); }
+  Status Close() override { return target_->Close(); }
+  Status Abort() override { return target_->Abort(); }
+  Result<int64_t> Tell() const override { return target_->Tell(); }
+  bool closed() const override { return target_->closed(); }
+  Result<int64_t> Read(int64_t nbytes, void* out) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes, out);
+  }
+  Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes);
+  }
+  Status Seek(int64_t pos) { return target_->Seek(pos); }
+
+ private:
+  std::shared_ptr<io::BufferReader> target_;
+  double latency_s_;
+};
+
+static ReadOptions CreateReadOptions(bool use_threads, bool use_async) {
+  auto result = csv::ReadOptions::Defaults();
+  result.use_threads = use_threads;
+  result.legacy_blocking_reads = !use_async;
+  // Simulate larger files by using smaller block files so the impact of multiple
+  // blocks is seen but we don't have to spend the time waiting on the large I/O
+  result.block_size = (1 << 20) / 100;
+  return result;
+}
+
+static std::shared_ptr<SlowInputStream> CreateStreamReader(std::shared_ptr<Buffer> buffer,
+                                                           int64_t latency_ms) {
+  auto buffer_reader = std::make_shared<io::BufferReader>(buffer);
+  return std::make_shared<SlowInputStream>(buffer_reader, latency_ms);
+}
+
+static void BenchmarkReader(benchmark::State& state, bool use_threads, bool use_async) {
+  auto latency_ms = state.range(0);
+  auto num_rows = state.range(1);
+  auto num_files = state.range(2);
+  if (num_files > 5 && use_threads && !use_async) {

Review comment:
       Why `5`? Is it related to the thread pool size below?

##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NTASKS = 100;
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+
+  std::vector<Future<std::shared_ptr<Table>>> task_futures(NTASKS);
+  for (int i = 0; i < NTASKS; i++) {
+    auto input = std::make_shared<io::BufferReader>(table_buffer);
+    ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+    task_futures[i] = reader->ReadAsync();
+  }
+  auto combined_future = All(task_futures);
+  combined_future.Wait();
+
+  ASSERT_OK_AND_ASSIGN(std::vector<Result<std::shared_ptr<Table>>> results,
+                       combined_future.result());
+  for (auto&& result : results) {
+    ASSERT_OK_AND_ASSIGN(auto table, result);
+    ASSERT_EQ(NROWS, table->num_rows());
+  }
+}
+
+void TestNestedParallelism(
+    std::shared_ptr<internal::ThreadPool> thread_pool,
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+  auto input = std::make_shared<io::BufferReader>(table_buffer);
+  ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+
+  Future<std::shared_ptr<Table>> table_future;
+
+  auto read_task = [&reader, &table_future]() mutable {
+    table_future = reader->ReadAsync();
+    return Status::OK();
+  };
+  ASSERT_OK_AND_ASSIGN(auto future, thread_pool->Submit(read_task));
+  ASSERT_TRUE(future.Wait(1));
+
+  if (future.is_finished()) {
+    ASSERT_TRUE(table_future.Wait(1));
+    if (table_future.is_finished()) {

Review comment:
       Same question here.

##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,9 +832,203 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {

Review comment:
       It would seem more robust and readable to test for the `result()` below.

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +615,74 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>
+Future<BreakValueType> Loop(Iterate iterate) {
+  auto break_fut = Future<BreakValueType>::Make();
+
+  struct Callback {
+    bool CheckForTermination(const Result<Control>& maybe_control) {
+      if (!maybe_control.ok() || maybe_control->IsBreak()) {
+        Result<BreakValueType> maybe_break = maybe_control.Map(Control::MoveBreakValue);
+        break_fut.MarkFinished(std::move(maybe_break));
+        return true;
+      }
+      return false;
+    }
+
+    void operator()(const Result<Control>& maybe_control) && {
+      if (CheckForTermination(maybe_control)) return;
+
+      auto control_fut = iterate();
+      while (control_fut.is_finished()) {
+        // There's no need to AddCallback on a finished future; we can CheckForTermination
+        // now. This also avoids recursion and potential stack overflow.
+        if (CheckForTermination(control_fut.result())) return;
+
+        control_fut = iterate();
+      }
+      control_fut.AddCallback(std::move(*this));
+    }
+
+    Iterate iterate;
+    // If the future returned by control_fut is never completed then we will be hanging on
+    // to break_fut forever even if the listener has given up listening on it.  Instead we
+    // rely on the fact that a producer (the caller of Future<>::Make) is always
+    // responsible for completing the futures they create.
+    // TODO: Could avoid this kind of situation with "future abandonment" similar to mesos
+    Future<BreakValueType> break_fut;

Review comment:
       Can't we simply use `WeakFuture` to avoid the aforementioned problem?

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}
+
+TEST(TestAsyncIteratorTransform, SkipSome) {
+  auto original = AsyncVectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = TransformAsyncGenerator(std::move(original), filter);
+  AssertAsyncGeneratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestAsyncUtil, ReadaheadFailed) {
+  auto source = []() -> Future<TestInt> {
+    return Future<TestInt>::MakeFinished(Status::Invalid("X"));
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  auto next = readahead();
+  ASSERT_EQ(Status::Invalid("X"), next.status());
+}
+
+TEST(TestAsyncUtil, Readahead) {
+  int num_delivered = 0;
+  auto source = [&num_delivered]() {
+    if (num_delivered < 5) {
+      return Future<TestInt>::MakeFinished(num_delivered++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  // Should not pump until first item requested

Review comment:
       Sounds weird. Is there a motivation for this?

##########
File path: cpp/src/arrow/util/task_group.cc
##########
@@ -135,6 +149,18 @@ class ThreadedTaskGroup : public TaskGroup {
       // before cv.notify_one() has returned
       std::unique_lock<std::mutex> lock(mutex_);
       cv_.notify_one();
+      if (completion_future_.has_value()) {
+        // MarkFinished could be slow.  We don't want to call it while we are holding
+        // the lock.
+        // TODO: If optional is thread safe then we can skip this locking entirely
+        auto future = *completion_future_;

Review comment:
       `auto&`?

##########
File path: cpp/src/arrow/util/task_group.cc
##########
@@ -135,6 +149,18 @@ class ThreadedTaskGroup : public TaskGroup {
       // before cv.notify_one() has returned
       std::unique_lock<std::mutex> lock(mutex_);
       cv_.notify_one();
+      if (completion_future_.has_value()) {
+        // MarkFinished could be slow.  We don't want to call it while we are holding
+        // the lock.
+        // TODO: If optional is thread safe then we can skip this locking entirely
+        auto future = *completion_future_;
+        auto finished = completion_future_->is_finished();
+        auto status = status_;
+        lock.unlock();
+        if (!finished) {
+          future.MarkFinished(status);

Review comment:
       It seems it is possible for `MarkFinished` to be called from several threads at once here, but it's not thread-safe.

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.
+template <typename T>
+class BackgroundIterator {
+  using PromiseType = typename detail::BackgroundIteratorPromise<T>;
+
+ public:
+  explicit BackgroundIterator(Iterator<T> it, internal::Executor* executor)
+      : it_(new Iterator<T>(std::move(it))),
+        queue_(new detail::ReadaheadQueue(0)),
+        executor_(executor),
+        done_() {}
+
+  ~BackgroundIterator() {
+    if (queue_) {
+      // Make sure the queue doesn't call any promises after this object
+      // is destroyed.
+      queue_->EnsureShutdownOrDie();
+    }
+  }
+
+  ARROW_DEFAULT_MOVE_AND_ASSIGN(BackgroundIterator);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(BackgroundIterator);
+
+  Future<T> operator()() {
+    if (done_) {
+      return Future<T>::MakeFinished(IterationTraits<T>::End());
+    }
+    auto promise = std::unique_ptr<PromiseType>(new PromiseType{it_.get()});
+    auto result = Future<T>(promise->out_);

Review comment:
       `result` is a bit misleading...

##########
File path: cpp/src/arrow/csv/reader_benchmark.cc
##########
@@ -0,0 +1,139 @@
+// 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 <string>
+
+#include "arrow/buffer.h"
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+class SlowInputStream : public io::InputStream {
+ public:
+  explicit SlowInputStream(std::shared_ptr<io::BufferReader> target, int64_t latency_ms)
+      : target_(std::move(target)) {
+    latency_s_ = static_cast<double>(latency_ms) / 1000.0;
+  }
+  virtual ~SlowInputStream() {}
+
+  Result<util::string_view> Peek(int64_t nbytes) override {
+    return target_->Peek(nbytes);
+  }
+  bool supports_zero_copy() const override { return target_->supports_zero_copy(); }
+  Status Close() override { return target_->Close(); }
+  Status Abort() override { return target_->Abort(); }
+  Result<int64_t> Tell() const override { return target_->Tell(); }
+  bool closed() const override { return target_->closed(); }
+  Result<int64_t> Read(int64_t nbytes, void* out) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes, out);
+  }
+  Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes);
+  }
+  Status Seek(int64_t pos) { return target_->Seek(pos); }
+
+ private:
+  std::shared_ptr<io::BufferReader> target_;
+  double latency_s_;
+};
+
+static ReadOptions CreateReadOptions(bool use_threads, bool use_async) {
+  auto result = csv::ReadOptions::Defaults();
+  result.use_threads = use_threads;
+  result.legacy_blocking_reads = !use_async;
+  // Simulate larger files by using smaller block files so the impact of multiple
+  // blocks is seen but we don't have to spend the time waiting on the large I/O
+  result.block_size = (1 << 20) / 100;
+  return result;
+}
+
+static std::shared_ptr<SlowInputStream> CreateStreamReader(std::shared_ptr<Buffer> buffer,
+                                                           int64_t latency_ms) {
+  auto buffer_reader = std::make_shared<io::BufferReader>(buffer);
+  return std::make_shared<SlowInputStream>(buffer_reader, latency_ms);
+}
+
+static void BenchmarkReader(benchmark::State& state, bool use_threads, bool use_async) {
+  auto latency_ms = state.range(0);
+  auto num_rows = state.range(1);
+  auto num_files = state.range(2);
+  if (num_files > 5 && use_threads && !use_async) {
+    state.SkipWithError("Would deadlock");
+  }
+  auto input_buffer = *MakeSampleCsvBuffer(num_rows);
+  // Hard coding # of threads so we don't deadlock if there are too few cores
+  ASSIGN_OR_ABORT(auto thread_pool, internal::ThreadPool::Make(6));
+  io::AsyncContext async_context(thread_pool.get());
+  while (state.KeepRunning()) {
+    std::vector<Future<std::shared_ptr<Table>>> table_futures;
+    for (int i = 0; i < num_files; i++) {
+      auto stream_reader = CreateStreamReader(input_buffer, latency_ms);
+      auto table_reader = *csv::TableReader::Make(
+          default_memory_pool(), async_context, stream_reader,
+          CreateReadOptions(use_threads, use_async), csv::ParseOptions::Defaults(),
+          csv::ConvertOptions::Defaults());
+      if (use_async) {
+        table_futures.push_back(table_reader->ReadAsync());
+      } else {
+        ASSERT_OK_AND_ASSIGN(auto table_future,
+                             async_context.executor->Submit(
+                                 [table_reader] { return table_reader->Read(); }));
+        table_futures.push_back(table_future);
+      }
+    }
+    auto combined = All(table_futures);
+    ASSIGN_OR_ABORT(auto result, combined.result());
+    for (auto&& table : result) {
+      ABORT_NOT_OK(table);
+    }
+  }
+  state.SetItemsProcessed(state.iterations() * num_rows);

Review comment:
       Do we want to also multiply by `num_files`?

##########
File path: cpp/src/arrow/csv/test_common.cc
##########
@@ -61,5 +61,47 @@ void MakeColumnParser(std::vector<std::string> items, std::shared_ptr<BlockParse
   ASSERT_EQ((*out)->num_rows(), items.size());
 }
 
+const std::vector<std::string> int64_rows = {"123", "4", "-317005557", "", "N/A", "0"};
+const std::vector<std::string> float_rows = {"0", "123.456", "-3170.55766", "", "N/A"};
+const std::vector<std::string> decimal128_rows = {"0", "123.456", "-3170.55766",
+                                                  "",  "N/A",     "1233456789.123456789"};
+const std::vector<std::string> iso8601_rows = {"1917-10-17", "2018-09-13",
+                                               "1941-06-22 04:00", "1945-05-09 09:45:38"};
+const std::vector<std::string> strptime_rows = {"10/17/1917", "9/13/2018", "9/5/1945"};

Review comment:
       Unless you're passing an explicit schema, the decimal128 and strptime rows won't be type-inferred as you seem to expect them to?

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {

Review comment:
       Iterator transform tests should use two different types for `T` and `V` (for example `int` and `std::string`), IMHO, to make sure that there's no place where the original value is passed through by mistake.

##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NTASKS = 100;
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+
+  std::vector<Future<std::shared_ptr<Table>>> task_futures(NTASKS);
+  for (int i = 0; i < NTASKS; i++) {
+    auto input = std::make_shared<io::BufferReader>(table_buffer);
+    ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+    task_futures[i] = reader->ReadAsync();
+  }
+  auto combined_future = All(task_futures);
+  combined_future.Wait();
+
+  ASSERT_OK_AND_ASSIGN(std::vector<Result<std::shared_ptr<Table>>> results,
+                       combined_future.result());
+  for (auto&& result : results) {
+    ASSERT_OK_AND_ASSIGN(auto table, result);
+    ASSERT_EQ(NROWS, table->num_rows());
+  }
+}
+
+void TestNestedParallelism(
+    std::shared_ptr<internal::ThreadPool> thread_pool,
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+  auto input = std::make_shared<io::BufferReader>(table_buffer);
+  ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+
+  Future<std::shared_ptr<Table>> table_future;
+
+  auto read_task = [&reader, &table_future]() mutable {
+    table_future = reader->ReadAsync();
+    return Status::OK();
+  };
+  ASSERT_OK_AND_ASSIGN(auto future, thread_pool->Submit(read_task));
+  ASSERT_TRUE(future.Wait(1));
+
+  if (future.is_finished()) {

Review comment:
       Hmm... what if it's not finished? Does it mean we should error out?

##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {

Review comment:
       Hide the `std::function<...>` behing a type alias (`using TableReaderFactory = ...`) to avoid repeating it several times?

##########
File path: cpp/src/arrow/csv/test_common.cc
##########
@@ -61,5 +61,47 @@ void MakeColumnParser(std::vector<std::string> items, std::shared_ptr<BlockParse
   ASSERT_EQ((*out)->num_rows(), items.size());
 }
 
+const std::vector<std::string> int64_rows = {"123", "4", "-317005557", "", "N/A", "0"};
+const std::vector<std::string> float_rows = {"0", "123.456", "-3170.55766", "", "N/A"};
+const std::vector<std::string> decimal128_rows = {"0", "123.456", "-3170.55766",
+                                                  "",  "N/A",     "1233456789.123456789"};
+const std::vector<std::string> iso8601_rows = {"1917-10-17", "2018-09-13",
+                                               "1941-06-22 04:00", "1945-05-09 09:45:38"};
+const std::vector<std::string> strptime_rows = {"10/17/1917", "9/13/2018", "9/5/1945"};
+
+static void WriteHeader(std::ostream& writer) {
+  writer << "Int64,Float,Decimal128,ISO8601,Strptime" << std::endl;
+}
+
+static std::string GetCell(std::vector<std::string> base_rows, size_t row_index) {

Review comment:
       Nit, but a const-ref is more efficient here.

##########
File path: cpp/src/arrow/csv/test_common.cc
##########
@@ -61,5 +61,47 @@ void MakeColumnParser(std::vector<std::string> items, std::shared_ptr<BlockParse
   ASSERT_EQ((*out)->num_rows(), items.size());
 }
 
+const std::vector<std::string> int64_rows = {"123", "4", "-317005557", "", "N/A", "0"};

Review comment:
       Everything that's not exposed in a `.h` should be enclosed in the anonymous namespace, to miminize exported symbols.

##########
File path: cpp/src/arrow/csv/test_common.h
##########
@@ -22,6 +22,7 @@
 #include <vector>
 
 #include "arrow/csv/parser.h"
+#include "arrow/io/memory.h"

Review comment:
       You don't need this include in this `.h` file.

##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NTASKS = 100;
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+
+  std::vector<Future<std::shared_ptr<Table>>> task_futures(NTASKS);
+  for (int i = 0; i < NTASKS; i++) {
+    auto input = std::make_shared<io::BufferReader>(table_buffer);
+    ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+    task_futures[i] = reader->ReadAsync();
+  }
+  auto combined_future = All(task_futures);
+  combined_future.Wait();
+
+  ASSERT_OK_AND_ASSIGN(std::vector<Result<std::shared_ptr<Table>>> results,
+                       combined_future.result());
+  for (auto&& result : results) {
+    ASSERT_OK_AND_ASSIGN(auto table, result);
+    ASSERT_EQ(NROWS, table->num_rows());
+  }
+}
+
+void TestNestedParallelism(
+    std::shared_ptr<internal::ThreadPool> thread_pool,
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+  auto input = std::make_shared<io::BufferReader>(table_buffer);
+  ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+
+  Future<std::shared_ptr<Table>> table_future;
+
+  auto read_task = [&reader, &table_future]() mutable {
+    table_future = reader->ReadAsync();
+    return Status::OK();
+  };
+  ASSERT_OK_AND_ASSIGN(auto future, thread_pool->Submit(read_task));
+  ASSERT_TRUE(future.Wait(1));
+
+  if (future.is_finished()) {
+    ASSERT_TRUE(table_future.Wait(1));
+    if (table_future.is_finished()) {
+      ASSERT_OK_AND_ASSIGN(auto table, table_future.result());
+      ASSERT_EQ(table->num_rows(), NROWS);
+    }
+  }
+}  // namespace csv
+
+TEST(SerialReaderTests, Stress) {
+  auto task_factory = [](std::shared_ptr<io::InputStream> input_stream) {
+    return TableReader::Make(default_memory_pool(), io::AsyncContext(), input_stream,
+                             ReadOptions::Defaults(), ParseOptions::Defaults(),
+                             ConvertOptions::Defaults());
+  };
+  StressTableReader(task_factory);
+}
+
+TEST(SerialReaderTests, NestedParallelism) {
+  ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1));

Review comment:
       This is testing a harmless kind of nested parallelism with two different thread pools, right?
   (the one instantiated above and the one used by default for CPU tasks)

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -837,20 +885,121 @@ class ThreadedTableReader : public BaseTableReader {
   }
 
  protected:
-  ThreadPool* thread_pool_;
+  Executor* thread_pool_;
+  Iterator<std::shared_ptr<Buffer>> buffer_iterator_;
+};
+
+class AsyncThreadedTableReader
+    : public BaseTableReader,
+      public std::enable_shared_from_this<AsyncThreadedTableReader> {
+ public:
+  using BaseTableReader::BaseTableReader;
+
+  AsyncThreadedTableReader(MemoryPool* pool, std::shared_ptr<io::InputStream> input,
+                           const ReadOptions& read_options,
+                           const ParseOptions& parse_options,
+                           const ConvertOptions& convert_options, Executor* thread_pool)
+      : BaseTableReader(pool, input, read_options, parse_options, convert_options),
+        thread_pool_(thread_pool) {}
+
+  ~AsyncThreadedTableReader() override {
+    if (task_group_) {
+      // In case of error, make sure all pending tasks are finished before
+      // we start destroying BaseTableReader members
+      ARROW_UNUSED(task_group_->Finish());
+    }
+  }
+
+  Status Init() override {
+    ARROW_ASSIGN_OR_RAISE(auto istream_it,
+                          io::MakeInputStreamIterator(input_, read_options_.block_size));
+
+    ARROW_ASSIGN_OR_RAISE(auto bg_it,
+                          MakeBackgroundIterator(std::move(istream_it), thread_pool_));
+
+    int32_t block_queue_size = thread_pool_->GetCapacity();
+    auto rh_it = AddReadahead(bg_it, block_queue_size);
+    buffer_generator_ = CSVBufferIterator::MakeAsync(std::move(rh_it));
+    return Status::OK();
+  }
+
+  Result<std::shared_ptr<Table>> Read() override { return ReadAsync().result(); }
+
+  Future<std::shared_ptr<Table>> ReadAsync() override {
+    task_group_ = internal::TaskGroup::MakeThreaded(thread_pool_);

Review comment:
       Do you have plans to remove TaskGroup from the picture? Ideally we should be able to call `Future::Then` on an executor, or something similar?

##########
File path: cpp/src/arrow/result.h
##########
@@ -331,6 +332,7 @@ class ARROW_MUST_USE_TYPE Result : public util::EqualityComparable<Result<T>> {
     return ValueUnsafe();
   }
   T& operator*() & { return ValueOrDie(); }
+  T* operator->() { return &ValueOrDie(); }

Review comment:
       I don't know if it makes sense to add a rvalue variant below?

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -129,11 +130,44 @@ template <typename T>
 inline Iterator<T> EmptyIt() {
   return MakeEmptyIterator<T>();
 }
-
 inline Iterator<TestInt> VectorIt(std::vector<TestInt> v) {
   return MakeVectorIterator<TestInt>(std::move(v));
 }
 
+std::function<Future<TestInt>()> AsyncVectorIt(std::vector<TestInt> v) {

Review comment:
       Return `AsyncGenerator<TestInt>`?

##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NTASKS = 100;
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+
+  std::vector<Future<std::shared_ptr<Table>>> task_futures(NTASKS);
+  for (int i = 0; i < NTASKS; i++) {
+    auto input = std::make_shared<io::BufferReader>(table_buffer);
+    ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+    task_futures[i] = reader->ReadAsync();
+  }
+  auto combined_future = All(task_futures);
+  combined_future.Wait();
+
+  ASSERT_OK_AND_ASSIGN(std::vector<Result<std::shared_ptr<Table>>> results,
+                       combined_future.result());
+  for (auto&& result : results) {
+    ASSERT_OK_AND_ASSIGN(auto table, result);
+    ASSERT_EQ(NROWS, table->num_rows());
+  }
+}
+
+void TestNestedParallelism(
+    std::shared_ptr<internal::ThreadPool> thread_pool,
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+  auto input = std::make_shared<io::BufferReader>(table_buffer);
+  ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+
+  Future<std::shared_ptr<Table>> table_future;
+
+  auto read_task = [&reader, &table_future]() mutable {
+    table_future = reader->ReadAsync();
+    return Status::OK();
+  };
+  ASSERT_OK_AND_ASSIGN(auto future, thread_pool->Submit(read_task));
+  ASSERT_TRUE(future.Wait(1));
+
+  if (future.is_finished()) {
+    ASSERT_TRUE(table_future.Wait(1));
+    if (table_future.is_finished()) {
+      ASSERT_OK_AND_ASSIGN(auto table, table_future.result());
+      ASSERT_EQ(table->num_rows(), NROWS);
+    }
+  }
+}  // namespace csv
+
+TEST(SerialReaderTests, Stress) {
+  auto task_factory = [](std::shared_ptr<io::InputStream> input_stream) {
+    return TableReader::Make(default_memory_pool(), io::AsyncContext(), input_stream,
+                             ReadOptions::Defaults(), ParseOptions::Defaults(),
+                             ConvertOptions::Defaults());
+  };
+  StressTableReader(task_factory);
+}
+
+TEST(SerialReaderTests, NestedParallelism) {
+  ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1));
+  auto task_factory = [](std::shared_ptr<io::InputStream> input_stream) {
+    return TableReader::Make(default_memory_pool(), io::AsyncContext(), input_stream,
+                             ReadOptions::Defaults(), ParseOptions::Defaults(),
+                             ConvertOptions::Defaults());
+  };
+  TestNestedParallelism(thread_pool, task_factory);
+}
+
+TEST(ThreadedReaderTests, Stress) {
+  ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1));
+  auto task_factory = [&thread_pool](std::shared_ptr<io::InputStream> input_stream)
+      -> Result<std::shared_ptr<TableReader>> {
+    ReadOptions read_options = ReadOptions::Defaults();
+    read_options.use_threads = true;
+    read_options.legacy_blocking_reads = true;
+    auto table_reader = TableReader::Make(
+        default_memory_pool(), io::AsyncContext(thread_pool.get()), input_stream,

Review comment:
       I'm not sure what difference it makes to change the executor used by the `AsyncContext`?
   (in any case it's an executor for IO tasks, which should be distinct from the executor for CPU tasks)

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -556,6 +559,38 @@ inline bool WaitForAll(const std::vector<Future<T>*>& futures,
   return waiter->Wait(seconds);
 }
 
+/// \brief Create a Future which completes when all of `futures` complete.
+///
+/// The future's result is a vector of the results of `futures`.
+/// Note that this future will never be marked "failed"; failed results
+/// will be stored in the result vector alongside successful results.
+template <typename T>
+Future<std::vector<Result<T>>> All(std::vector<Future<T>> futures) {
+  struct State {
+    explicit State(std::vector<Future<T>> f)
+        : futures(std::move(f)), n_remaining(futures.size()) {}
+
+    std::vector<Future<T>> futures;
+    std::atomic<size_t> n_remaining;
+  };

Review comment:
       Hmm... did you investigate whether it was possible to reuse `FutureWaiter` instead of recreating a similar logic here?

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"

Review comment:
       Please check whether all these includes are necessary. At least `thread_pool.h` doesn't seem necessary, unless I'm mistaken.

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +187,122 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_ = false;
+  bool ready_for_next_ = false;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {

Review comment:
       What is "ready_for_next"?

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {

Review comment:
       Is this supposed to be a public method?

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {

Review comment:
       Since we don't know how much state the generator and visitor carry, I think it would be better to move-construct an anonymous callable object.

##########
File path: cpp/src/arrow/csv/reader_benchmark.cc
##########
@@ -0,0 +1,139 @@
+// 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 <string>
+
+#include "arrow/buffer.h"
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+class SlowInputStream : public io::InputStream {
+ public:
+  explicit SlowInputStream(std::shared_ptr<io::BufferReader> target, int64_t latency_ms)
+      : target_(std::move(target)) {
+    latency_s_ = static_cast<double>(latency_ms) / 1000.0;
+  }
+  virtual ~SlowInputStream() {}
+
+  Result<util::string_view> Peek(int64_t nbytes) override {
+    return target_->Peek(nbytes);
+  }
+  bool supports_zero_copy() const override { return target_->supports_zero_copy(); }
+  Status Close() override { return target_->Close(); }
+  Status Abort() override { return target_->Abort(); }
+  Result<int64_t> Tell() const override { return target_->Tell(); }
+  bool closed() const override { return target_->closed(); }
+  Result<int64_t> Read(int64_t nbytes, void* out) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes, out);
+  }
+  Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes);
+  }
+  Status Seek(int64_t pos) { return target_->Seek(pos); }
+
+ private:
+  std::shared_ptr<io::BufferReader> target_;
+  double latency_s_;
+};
+
+static ReadOptions CreateReadOptions(bool use_threads, bool use_async) {
+  auto result = csv::ReadOptions::Defaults();
+  result.use_threads = use_threads;
+  result.legacy_blocking_reads = !use_async;
+  // Simulate larger files by using smaller block files so the impact of multiple
+  // blocks is seen but we don't have to spend the time waiting on the large I/O
+  result.block_size = (1 << 20) / 100;
+  return result;
+}
+
+static std::shared_ptr<SlowInputStream> CreateStreamReader(std::shared_ptr<Buffer> buffer,
+                                                           int64_t latency_ms) {
+  auto buffer_reader = std::make_shared<io::BufferReader>(buffer);
+  return std::make_shared<SlowInputStream>(buffer_reader, latency_ms);
+}
+
+static void BenchmarkReader(benchmark::State& state, bool use_threads, bool use_async) {
+  auto latency_ms = state.range(0);
+  auto num_rows = state.range(1);
+  auto num_files = state.range(2);
+  if (num_files > 5 && use_threads && !use_async) {
+    state.SkipWithError("Would deadlock");
+  }
+  auto input_buffer = *MakeSampleCsvBuffer(num_rows);
+  // Hard coding # of threads so we don't deadlock if there are too few cores

Review comment:
       Can you make it a `constexpr something` at least?
   
   Also, why would we deadlock? The implementation should certainly prevent that. The user may be running in a 1- or 2-core VM.

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {

Review comment:
       I'm skeptical about taking `std::function<>` and praying that the compiler optimizes common cases fine, instead of a callable template argument.

##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,9 +832,203 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {

Review comment:
       Same here.

##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,9 +832,203 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(3, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(Status::IOError("XYZ"), results[1].status());
+    ASSERT_EQ(3, *results[2]);
+  }));
+
+  f1.MarkFinished(1);
+  f2.MarkFinished(Status::IOError("XYZ"));
+  f3.MarkFinished(3);
+
+  AssertFinished(combined);
+}
+
+TEST(FutureLoopTest, Sync) {
+  struct {
+    int i = 0;
+    Future<int> Get() { return Future<int>::MakeFinished(i++); }
+  } IntSource;
+
+  bool do_fail = false;
+  std::vector<int> ints;
+  auto loop_body = [&] {
+    return IntSource.Get().Then([&](int i) -> Result<ControlFlow<int>> {
+      if (do_fail && i == 3) {
+        return Status::IOError("xxx");
+      }
+
+      if (i == 5) {
+        int sum = 0;
+        for (int i : ints) sum += i;
+        return Break(sum);
+      }
+
+      ints.push_back(i);
+      return Continue();
+    });
+  };
+
+  {
+    auto sum_fut = Loop(loop_body);
+    AssertSuccessful(sum_fut);
+
+    ASSERT_OK_AND_ASSIGN(auto sum, sum_fut.result());
+    ASSERT_EQ(sum, 0 + 1 + 2 + 3 + 4);
+  }
+
+  {
+    do_fail = true;
+    IntSource.i = 0;
+    auto sum_fut = Loop(loop_body);
+    AssertFailed(sum_fut);
+    ASSERT_RAISES(IOError, sum_fut.result());
+  }
+}
+
+TEST(FutureLoopTest, EmptyBreakValue) {
+  Future<> none_fut =
+      Loop([&] { return Future<>::MakeFinished().Then([&](...) { return Break(); }); });
+  AssertSuccessful(none_fut);
+}
+
+TEST(FutureLoopTest, MoveOnlyBreakValue) {
+  Future<MoveOnlyDataType> one_fut = Loop([&] {
+    return Future<int>::MakeFinished(1).Then(
+        [&](int i) { return Break(MoveOnlyDataType(i)); });
+  });
+  AssertSuccessful(one_fut);
+  ASSERT_OK_AND_ASSIGN(auto one, std::move(one_fut).result());
+  ASSERT_EQ(one, 1);
+}
+
+TEST(FutureLoopTest, StackOverflow) {
+  // Looping over futures is normally a rather recursive task.  If the futures complete
+  // synchronously (because they are already finished) it could lead to a stack overflow
+  // if care is not taken.
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 1000000) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());
+    }
+    return Future<ControlFlow<int>>::MakeFinished(Break(-1));
+  };
+  auto loop_fut = Loop(loop_body);
+  ASSERT_TRUE(loop_fut.Wait(0.1));
+}
+
+TEST(FutureLoopTest, AllowsBreakFutToBeDiscarded) {
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 10) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());
+    }
+    return Future<ControlFlow<int>>::MakeFinished(Break(-1));
+  };
+  auto loop_fut = Loop(loop_body).Then([](...) { return Status::OK(); });
+  ASSERT_TRUE(loop_fut.Wait(0.1));
+}
+
+TEST(FutureLoopTest, EmptyLoop) {
+  auto loop_body = []() -> Future<ControlFlow<int>> {
+    return Future<ControlFlow<int>>::MakeFinished(Break(0));
+  };
+  auto loop_fut = Loop(loop_body);
+  ASSERT_TRUE(loop_fut.Wait(0.1));
+  if (loop_fut.is_finished()) {

Review comment:
       Why the `if`?

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +615,74 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>
+Future<BreakValueType> Loop(Iterate iterate) {
+  auto break_fut = Future<BreakValueType>::Make();
+
+  struct Callback {
+    bool CheckForTermination(const Result<Control>& maybe_control) {
+      if (!maybe_control.ok() || maybe_control->IsBreak()) {
+        Result<BreakValueType> maybe_break = maybe_control.Map(Control::MoveBreakValue);
+        break_fut.MarkFinished(std::move(maybe_break));
+        return true;
+      }
+      return false;
+    }
+
+    void operator()(const Result<Control>& maybe_control) && {
+      if (CheckForTermination(maybe_control)) return;
+
+      auto control_fut = iterate();
+      while (control_fut.is_finished()) {
+        // There's no need to AddCallback on a finished future; we can CheckForTermination
+        // now. This also avoids recursion and potential stack overflow.

Review comment:
       But note the future may finish between here and the `AddCallback()` call below. So while this reduces the risk of stack overflow, it doesn't eliminate it.
   
   Is there a way to avoid such fragility?
   

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -428,10 +547,16 @@ struct ReadaheadIteratorPromise : ReadaheadPromise {
 
   explicit ReadaheadIteratorPromise(Iterator<T>* it) : it_(it) {}
 
-  void Call() override {
+  bool Call() override {
     assert(!called_);
     out_ = it_->Next();
     called_ = true;
+    return out_ == IterationTraits<T>::End();
+  }
+
+  void End() override {
+    // No need to do anything for the synchronous case.  No one is waiting on this
+    // called_ = true;

Review comment:
       Why is this commented out?

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +615,74 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>

Review comment:
       Hmm... can't `ControlFlow<T>` simply be a `util::optional<T>`?

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +187,122 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_ = false;
+  bool ready_for_next_ = false;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<Result<TransformFlow<V>>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  // Calls the transform function on the current value.  Can return in several ways
+  // * If the next value is requested (e.g. skip) it will return an empty optional
+  // * If an invalid status is encountered that will be returned
+  // * If finished it will return IterationTraits<V>::End()
+  // * If a value is returned by the transformer that will be returned
+  Result<util::optional<V>> Pump() {

Review comment:
       Is this public?

##########
File path: cpp/src/arrow/util/task_group.h
##########
@@ -63,6 +63,20 @@ class ARROW_EXPORT TaskGroup : public std::enable_shared_from_this<TaskGroup> {
   /// task (or subgroup).
   virtual Status Finish() = 0;
 
+  /// Returns a future that will complete the first time all tasks are finished.
+  /// This should be called only after all top level tasks
+  /// have been added to the task group.
+  ///
+  /// If you are using a TaskGroup asyncrhonously there are a few considerations to keep

Review comment:
       "asynchronously"

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {

Review comment:
       Why `if`?

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +187,122 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_ = false;
+  bool ready_for_next_ = false;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<Result<TransformFlow<V>>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  // Calls the transform function on the current value.  Can return in several ways
+  // * If the next value is requested (e.g. skip) it will return an empty optional
+  // * If an invalid status is encountered that will be returned
+  // * If finished it will return IterationTraits<V>::End()
+  // * If a value is returned by the transformer that will be returned
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      ARROW_ASSIGN_OR_RAISE(util::optional<V> next, Pump());
+      if (next.has_value()) {
+        return *next;
+      }
+      ARROW_ASSIGN_OR_RAISE(last_value_, it_.Next());
+    }
+    return IterationTraits<V>::End();
+  }
+
+ private:
+  Iterator<T> it_;
+  Transformer<T, V> transformer_;
+  util::optional<T> last_value_;
+  bool finished_ = false;
+};
+
+/// \brief Transforms an iterator according to a transformer, returning a new Iterator.
+///
+/// The transformer will be called on each element of the source iterator and for each
+/// call it can yield a value, skip, or finish the iteration.  When yielding a value the
+/// transformer can choose to consume the source item (the default, ready_for_next = true)
+/// or to keep it and it will be called again on the same value.

Review comment:
       Why called again on the same value? Is there some non-toy use case?

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {

Review comment:
       Can you take care to group the tests semantically?
   I would expect all `TestIteratorTransform`s to be grouped together.
   Also it would be nice to have async iterator tests grouped separately from regular iterator tests.
   Also, basic primitives (e.g. `CollectAsyncGenerator`) should generally be tested before more complex ones.
   
   Without any care for test ordering, test files become write-only.

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);

Review comment:
       Hmm... unless I'm mistaken, `vector` should be a `Result<std::vector<TestInt>>`? How can it be compared with a `std::vector<TestInt>`?

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}
+
+TEST(TestAsyncIteratorTransform, SkipSome) {
+  auto original = AsyncVectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = TransformAsyncGenerator(std::move(original), filter);
+  AssertAsyncGeneratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestAsyncUtil, ReadaheadFailed) {
+  auto source = []() -> Future<TestInt> {
+    return Future<TestInt>::MakeFinished(Status::Invalid("X"));
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  auto next = readahead();
+  ASSERT_EQ(Status::Invalid("X"), next.status());

Review comment:
       Should the next ones also fail? With the same error / different one?

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}

Review comment:
       Should the third one still succeed?

##########
File path: cpp/src/arrow/util/thread_pool.h
##########
@@ -87,6 +87,21 @@ class ARROW_EXPORT Executor {
     return SpawnReal(hints, std::forward<Function>(func));
   }
 
+  template <typename T>
+  Future<T> Transfer(Future<T> future) {

Review comment:
       Hmm... so the only thing that will be spawed on the executor (presumably a thread pool) is a `MarkFinished` called?
   I suppose the intent is so that `Then()` calls are also made on the executor?
   
   In any case, please add a comment or docstring describing this and the intent.

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous

Review comment:
       Then replace it with `ASSERT_TRUE(sum_future.finished())`?

##########
File path: cpp/src/arrow/util/thread_pool.h
##########
@@ -87,6 +87,21 @@ class ARROW_EXPORT Executor {
     return SpawnReal(hints, std::forward<Function>(func));
   }
 
+  template <typename T>
+  Future<T> Transfer(Future<T> future) {
+    auto transferred = Future<T>::Make();
+    future.AddCallback([this, transferred](const Result<T>& result) mutable {
+      Result<T> result_copy(result);
+      auto spawn_status = Spawn([transferred, result_copy]() mutable {
+        transferred.MarkFinished(result_copy);

Review comment:
       and this is a third one?

##########
File path: cpp/src/arrow/util/task_group.cc
##########
@@ -135,6 +149,18 @@ class ThreadedTaskGroup : public TaskGroup {
       // before cv.notify_one() has returned
       std::unique_lock<std::mutex> lock(mutex_);
       cv_.notify_one();
+      if (completion_future_.has_value()) {
+        // MarkFinished could be slow.  We don't want to call it while we are holding
+        // the lock.
+        // TODO: If optional is thread safe then we can skip this locking entirely
+        auto future = *completion_future_;
+        auto finished = completion_future_->is_finished();
+        auto status = status_;

Review comment:
       `const auto&`?

##########
File path: cpp/src/arrow/util/thread_pool.h
##########
@@ -87,6 +87,21 @@ class ARROW_EXPORT Executor {
     return SpawnReal(hints, std::forward<Function>(func));
   }
 
+  template <typename T>
+  Future<T> Transfer(Future<T> future) {
+    auto transferred = Future<T>::Make();
+    future.AddCallback([this, transferred](const Result<T>& result) mutable {
+      Result<T> result_copy(result);
+      auto spawn_status = Spawn([transferred, result_copy]() mutable {

Review comment:
       This is a second copy.

##########
File path: cpp/src/arrow/util/task_group.cc
##########
@@ -135,6 +149,18 @@ class ThreadedTaskGroup : public TaskGroup {
       // before cv.notify_one() has returned
       std::unique_lock<std::mutex> lock(mutex_);
       cv_.notify_one();
+      if (completion_future_.has_value()) {
+        // MarkFinished could be slow.  We don't want to call it while we are holding
+        // the lock.
+        // TODO: If optional is thread safe then we can skip this locking entirely

Review comment:
       Well, the locking is necessary anyway, so I'm not sure the TODO is very useful.

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -16,6 +16,7 @@
 // under the License.
 
 #include "arrow/util/iterator.h"
+#include "arrow/util/async_iterator.h"

Review comment:
       Nit: keep includes sorted in lexicographic order.

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(

Review comment:
       This seems unused?

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;

Review comment:
       `*std::move(next_result)`

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}

Review comment:
       `std::move(generator)` and `std::move(transformer)`

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();

Review comment:
       `std::move(maybe_next_result).ValueUnsafe()`?

##########
File path: cpp/src/arrow/util/thread_pool.h
##########
@@ -87,6 +87,21 @@ class ARROW_EXPORT Executor {
     return SpawnReal(hints, std::forward<Function>(func));
   }
 
+  template <typename T>
+  Future<T> Transfer(Future<T> future) {
+    auto transferred = Future<T>::Make();
+    future.AddCallback([this, transferred](const Result<T>& result) mutable {
+      Result<T> result_copy(result);

Review comment:
       This is a first copy.

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue

Review comment:
       Why not pump as soon as `AddReadahead` is called? It would be more consistent with `ReadaheadIterator`.

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {

Review comment:
       Will make a copy of `generator`.

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();

Review comment:
       Instead of this, use an anonymous callable object?

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);

Review comment:
       `*std::move(maybe_next)`?

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;

Review comment:
       `const auto&`

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.
+template <typename T>
+class BackgroundIterator {

Review comment:
       It's a bit confusing that it's named `BackgroundIterator` even though it's not an iterator... should it be `BackgroundGenerator`?

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.
+template <typename T>
+class BackgroundIterator {
+  using PromiseType = typename detail::BackgroundIteratorPromise<T>;
+
+ public:
+  explicit BackgroundIterator(Iterator<T> it, internal::Executor* executor)
+      : it_(new Iterator<T>(std::move(it))),
+        queue_(new detail::ReadaheadQueue(0)),
+        executor_(executor),
+        done_() {}
+
+  ~BackgroundIterator() {
+    if (queue_) {
+      // Make sure the queue doesn't call any promises after this object
+      // is destroyed.
+      queue_->EnsureShutdownOrDie();
+    }
+  }
+
+  ARROW_DEFAULT_MOVE_AND_ASSIGN(BackgroundIterator);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(BackgroundIterator);
+
+  Future<T> operator()() {
+    if (done_) {
+      return Future<T>::MakeFinished(IterationTraits<T>::End());
+    }
+    auto promise = std::unique_ptr<PromiseType>(new PromiseType{it_.get()});
+    auto result = Future<T>(promise->out_);
+    // TODO: Need a futuristic version of ARROW_RETURN_NOT_OK
+    auto append_status = queue_->Append(
+        static_cast<std::unique_ptr<detail::ReadaheadPromise>>(std::move(promise)));
+    if (!append_status.ok()) {
+      return Future<T>::MakeFinished(append_status);
+    }
+
+    result.AddCallback([this](const Result<T>& result) {
+      if (!result.ok() || result.ValueUnsafe() == IterationTraits<T>::End()) {
+        done_ = true;
+      }
+    });
+
+    return executor_->Transfer(result);

Review comment:
       Unless I'm misreading this, it seems only the trivial callback above is spawned on the executor?

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.

Review comment:
       "in an executor", rather?
   
   Can you document whether the underlying iterator may be called from several threads at once?

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {

Review comment:
       Instead of having the closure state spread around as (copied) local variables, perhaps create a dedicated callable object?

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);

Review comment:
       Why use `AddCallback` while this can simply be done synchronously above? This will add overhead.

##########
File path: cpp/src/arrow/util/task_group_test.cc
##########
@@ -244,6 +245,65 @@ void TestNoCopyTask(std::shared_ptr<TaskGroup> task_group) {
   ASSERT_EQ(0, *counter);
 }
 
+void TestFinishNotSticky(std::function<std::shared_ptr<TaskGroup>()> factory) {
+  // If a task is added that runs very quickly it might decrement the task counter back
+  // down to 0 and mark the completion future as complete before all tasks are added.
+  // The "finished future" of the task group could get stuck to complete.
+  const int NTASKS = 100;
+  for (int i = 0; i < NTASKS; ++i) {
+    auto task_group = factory();
+    // Add a task and let it complete
+    task_group->Append([] { return Status::OK(); });
+    // Wait a little bit, if the task group was going to lock the finish hopefully it
+    // would do so here while we wait
+    SleepFor(1e-2);
+
+    // Add a new task that will still be running
+    std::atomic<bool> ready(false);
+    std::mutex m;
+    std::condition_variable cv;
+    task_group->Append([&m, &cv, &ready] {
+      std::unique_lock<std::mutex> lk(m);
+      cv.wait(lk, [&ready] { return ready.load(); });
+      return Status::OK();
+    });
+
+    // Ensure task group not finished already
+    auto finished = task_group->FinishAsync();
+    ASSERT_FALSE(finished.is_finished());
+
+    std::unique_lock<std::mutex> lk(m);
+    ready = true;
+    lk.unlock();
+    cv.notify_one();
+
+    ASSERT_TRUE(finished.Wait(1));

Review comment:
       Should we check that `status()` here as well (and below)?

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -837,20 +885,121 @@ class ThreadedTableReader : public BaseTableReader {
   }
 
  protected:
-  ThreadPool* thread_pool_;
+  Executor* thread_pool_;
+  Iterator<std::shared_ptr<Buffer>> buffer_iterator_;
+};
+
+class AsyncThreadedTableReader
+    : public BaseTableReader,
+      public std::enable_shared_from_this<AsyncThreadedTableReader> {
+ public:
+  using BaseTableReader::BaseTableReader;
+
+  AsyncThreadedTableReader(MemoryPool* pool, std::shared_ptr<io::InputStream> input,
+                           const ReadOptions& read_options,
+                           const ParseOptions& parse_options,
+                           const ConvertOptions& convert_options, Executor* thread_pool)
+      : BaseTableReader(pool, input, read_options, parse_options, convert_options),
+        thread_pool_(thread_pool) {}
+
+  ~AsyncThreadedTableReader() override {
+    if (task_group_) {
+      // In case of error, make sure all pending tasks are finished before
+      // we start destroying BaseTableReader members
+      ARROW_UNUSED(task_group_->Finish());
+    }
+  }
+
+  Status Init() override {
+    ARROW_ASSIGN_OR_RAISE(auto istream_it,
+                          io::MakeInputStreamIterator(input_, read_options_.block_size));
+
+    ARROW_ASSIGN_OR_RAISE(auto bg_it,
+                          MakeBackgroundIterator(std::move(istream_it), thread_pool_));
+
+    int32_t block_queue_size = thread_pool_->GetCapacity();
+    auto rh_it = AddReadahead(bg_it, block_queue_size);
+    buffer_generator_ = CSVBufferIterator::MakeAsync(std::move(rh_it));
+    return Status::OK();
+  }
+
+  Result<std::shared_ptr<Table>> Read() override { return ReadAsync().result(); }
+
+  Future<std::shared_ptr<Table>> ReadAsync() override {
+    task_group_ = internal::TaskGroup::MakeThreaded(thread_pool_);
+
+    auto self = shared_from_this();
+    return ProcessFirstBuffer().Then([self](const std::shared_ptr<Buffer> first_buffer) {
+      auto block_generator = ThreadedBlockReader::MakeAsyncIterator(
+          self->buffer_generator_, MakeChunker(self->parse_options_),
+          std::move(first_buffer));
+
+      std::function<Status(util::optional<CSVBlock>)> block_visitor =
+          [self](util::optional<CSVBlock> maybe_block) -> Status {
+        DCHECK(!maybe_block->consume_bytes);

Review comment:
       So `maybe_block` always has a value...?

##########
File path: cpp/src/arrow/util/task_group_test.cc
##########
@@ -292,5 +352,25 @@ TEST(ThreadedTaskGroup, StressFailingTaskGroupLifetime) {
       [&] { return TaskGroup::MakeThreaded(thread_pool.get()); });
 }
 
+TEST(ThreadedTaskGroup, FinishNotSticky) {
+  std::shared_ptr<ThreadPool> thread_pool;
+  ASSERT_OK_AND_ASSIGN(thread_pool, ThreadPool::Make(16));
+
+  TestFinishNotSticky([&] { return TaskGroup::MakeThreaded(thread_pool.get()); });
+}
+
+TEST(ThreadedTaskGroup, FinishNeverStarted) {
+  std::shared_ptr<ThreadPool> thread_pool;
+  ASSERT_OK_AND_ASSIGN(thread_pool, ThreadPool::Make(4));
+  TestFinishNeverStarted(TaskGroup::MakeThreaded(thread_pool.get()));
+}
+
+TEST(ThreadedTaskGroup, FinishAlreadyCompleted) {
+  std::shared_ptr<ThreadPool> thread_pool;
+  ASSERT_OK_AND_ASSIGN(thread_pool, ThreadPool::Make(16));
+
+  TestFinishAlreadyCompleted([&] { return TaskGroup::MakeThreaded(thread_pool.get()); });
+}

Review comment:
       Shouldn't you similarly call those tests for `SerialTaskGroup`?

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}
+
+TEST(TestAsyncIteratorTransform, SkipSome) {
+  auto original = AsyncVectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = TransformAsyncGenerator(std::move(original), filter);
+  AssertAsyncGeneratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestAsyncUtil, ReadaheadFailed) {
+  auto source = []() -> Future<TestInt> {
+    return Future<TestInt>::MakeFinished(Status::Invalid("X"));
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  auto next = readahead();
+  ASSERT_EQ(Status::Invalid("X"), next.status());
+}
+
+TEST(TestAsyncUtil, Readahead) {
+  int num_delivered = 0;
+  auto source = [&num_delivered]() {
+    if (num_delivered < 5) {
+      return Future<TestInt>::MakeFinished(num_delivered++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  // Should not pump until first item requested
+  ASSERT_EQ(0, num_delivered);
+
+  auto first = readahead();
+  // At this point the pumping should have happened
+  ASSERT_EQ(5, num_delivered);
+  ASSERT_EQ(0, first.result()->value);

Review comment:
       `ASSERT_OK_AND_EQ`?

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -150,20 +154,21 @@ struct CSVBlock {
   std::function<Status(int64_t)> consume_bytes;
 };
 
+// This is an unfortunate side-effect of using optional<T> as the iterator in the
+// CSVBlock iterator.  We need to be able to compare with
+// IterationTraits<optional<T>>::End() and empty optionals will always compare true but
+// the optional copmarator won't compile if the underlying type isn't comparable

Review comment:
       "comparator"

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -220,14 +238,36 @@ class ThreadedBlockReader : public BlockReader {
  public:
   using BlockReader::BlockReader;
 
-  Result<arrow::util::optional<CSVBlock>> Next() {
+  static Iterator<util::optional<CSVBlock>> MakeIterator(
+      Iterator<std::shared_ptr<Buffer>> buffer_iterator, std::unique_ptr<Chunker> chunker,
+      std::shared_ptr<Buffer> first_buffer) {
+    auto block_reader =
+        std::make_shared<ThreadedBlockReader>(std::move(chunker), first_buffer);
+    // Wrap shared pointer in callable
+    Transformer<std::shared_ptr<Buffer>, util::optional<CSVBlock>> block_reader_fn =
+        [block_reader](std::shared_ptr<Buffer> next) { return (*block_reader)(next); };
+    return MakeTransformedIterator(std::move(buffer_iterator), block_reader_fn);
+  }
+
+  static AsyncGenerator<util::optional<CSVBlock>> MakeAsyncIterator(
+      AsyncGenerator<std::shared_ptr<Buffer>> buffer_generator,
+      std::unique_ptr<Chunker> chunker, std::shared_ptr<Buffer> first_buffer) {
+    auto block_reader =
+        std::make_shared<ThreadedBlockReader>(std::move(chunker), first_buffer);
+    // Wrap shared pointer in callable
+    Transformer<std::shared_ptr<Buffer>, util::optional<CSVBlock>> block_reader_fn =
+        [block_reader](std::shared_ptr<Buffer> next) { return (*block_reader)(next); };
+    return TransformAsyncGenerator(buffer_generator, block_reader_fn);

Review comment:
       `std::move(buffer_generator)`?

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());

Review comment:
       `ASSERT_OK_AND_EQ`?

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -177,14 +182,26 @@ class SerialBlockReader : public BlockReader {
  public:
   using BlockReader::BlockReader;
 
-  Result<arrow::util::optional<CSVBlock>> Next() {
+  static Iterator<util::optional<CSVBlock>> MakeIterator(
+      Iterator<std::shared_ptr<Buffer>> buffer_iterator, std::unique_ptr<Chunker> chunker,
+      std::shared_ptr<Buffer> first_buffer) {
+    auto block_reader =
+        std::make_shared<SerialBlockReader>(std::move(chunker), first_buffer);
+    // Wrap shared pointer in callable
+    Transformer<std::shared_ptr<Buffer>, util::optional<CSVBlock>> block_reader_fn =
+        [block_reader](std::shared_ptr<Buffer> buf) {
+          return (*block_reader)(std::move(buf));
+        };
+    return MakeTransformedIterator(std::move(buffer_iterator), block_reader_fn);
+  }
+
+  Result<TransformFlow<util::optional<CSVBlock>>> operator()(

Review comment:
       Hmm... why are we still returning `util::optional` if we never yield a value-less result?

##########
File path: cpp/src/arrow/util/iterator.cc
##########
@@ -119,14 +123,30 @@ class ReadaheadQueue::Impl : public std::enable_shared_from_this<ReadaheadQueue:
   void DoWork() {
     std::unique_lock<std::mutex> lock(mutex_);
     while (!please_shutdown_) {
-      while (static_cast<int64_t>(done_.size()) < max_readahead_ && todo_.size() > 0) {
+      while (todo_.size() > 0 &&
+             ((max_readahead_ <= 0) ||
+              (static_cast<int64_t>(done_.size()) < max_readahead_))) {
         auto promise = std::move(todo_.front());
         todo_.pop_front();
         lock.unlock();
-        promise->Call();
+        if (promise->Call()) {
+          // If the call finished then we should purge the remaining TODO items, marking
+          // them finished
+          lock.lock();
+          std::deque<std::unique_ptr<ReadaheadPromise>> to_clear(std::move(todo_));
+          // While the async iterator doesn't use todo_ anymore after it hits a finish the
+          // sync iterator might still due to timing so leave it valid
+          todo_.clear();
+          lock.unlock();
+          for (auto&& promise : to_clear) {
+            promise->End();
+          }
+        }
         lock.lock();
-        done_.push_back(std::move(promise));
-        work_done_.notify_one();
+        if (max_readahead_ > 0) {
+          done_.push_back(std::move(promise));
+          work_done_.notify_one();
+        }

Review comment:
       This is frankly weird. It seems the logic here is conflating "unbounded readahead" and "is an async generator".
   
   More generally, the fact that two different control flows (sync vs async) seem to be handled in the same implementation make things very confusing and difficult to follow. Can you find a way to separate the two cases?
   

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {

Review comment:
       (same for async generator transform tests)




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



[GitHub] [arrow] westonpace commented on pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-777175290


   @pitrou It now runs on the CPU thread pool.  The I/O thread pool is sent down all the way into the table reader but is not yet used (I added ARROW-11590 for this).  When I tried to run it on the I/O thread pool it failed because there were multiple threads running the iterator.  The current behavior (dedicated single thread pool) matches the old behavior though so this isn't a regression.


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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r555266265



##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,6 +832,145 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(3, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(Status::IOError("XYZ"), results[1].status());
+    ASSERT_EQ(3, *results[2]);
+  }));
+
+  f1.MarkFinished(1);
+  f2.MarkFinished(Status::IOError("XYZ"));
+  f3.MarkFinished(3);
+
+  AssertFinished(combined);
+}
+
+TEST(FutureLoopTest, Sync) {
+  struct {
+    int i = 0;
+    Future<int> Get() { return Future<int>::MakeFinished(i++); }
+  } IntSource;
+
+  bool do_fail = false;
+  std::vector<int> ints;
+  auto loop_body = [&] {
+    return IntSource.Get().Then([&](int i) -> Result<ControlFlow<int>> {
+      if (do_fail && i == 3) {
+        return Status::IOError("xxx");
+      }
+
+      if (i == 5) {
+        int sum = 0;
+        for (int i : ints) sum += i;
+        return Break(sum);
+      }
+
+      ints.push_back(i);
+      return Continue();
+    });
+  };
+
+  {
+    auto sum_fut = Loop(loop_body);
+    AssertSuccessful(sum_fut);
+
+    ASSERT_OK_AND_ASSIGN(auto sum, sum_fut.result());
+    ASSERT_EQ(sum, 0 + 1 + 2 + 3 + 4);
+  }
+
+  {
+    do_fail = true;
+    IntSource.i = 0;
+    auto sum_fut = Loop(loop_body);
+    AssertFailed(sum_fut);
+    ASSERT_RAISES(IOError, sum_fut.result());
+  }
+}
+
+TEST(FutureLoopTest, EmptyBreakValue) {
+  Future<> none_fut =
+      Loop([&] { return Future<>::MakeFinished().Then([&](...) { return Break(); }); });
+  AssertSuccessful(none_fut);
+}
+
+TEST(FutureLoopTest, MoveOnlyBreakValue) {
+  Future<MoveOnlyDataType> one_fut = Loop([&] {
+    return Future<int>::MakeFinished(1).Then(
+        [&](int i) { return Break(MoveOnlyDataType(i)); });
+  });
+  AssertSuccessful(one_fut);
+  ASSERT_OK_AND_ASSIGN(auto one, std::move(one_fut).result());
+  ASSERT_EQ(one, 1);
+}
+
+TEST(FutureLoopTest, StackOverflow) {
+  // Looping over futures is normally a rather recursive task.  If the futures complete
+  // synchronously (because they are already finished) it could lead to a stack overflow
+  // if care is not taken.
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 1000000) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());

Review comment:
       Hmm...I'm a little torn here.  Could this make return type inference even harder?  For example, if you change
   
   ```
   auto some_future = [] {
     if (some_base_case) {
       return Future<T>::MakeFinished(0);
     }
     return SomeAsyncFunction.Then(...);
   };
   ```
   
   to...
   
   ```
   auto some_future = [] {
     if (some_base_case) {
       return 0;
     }
     return SomeAsyncFunction.Then(...);
   };
   ```




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



[GitHub] [arrow] bkietz commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r576262763



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -512,7 +545,7 @@ class ARROW_MUST_USE_TYPE Future {
   FRIEND_TEST(FutureRefTest, ChainRemoved);
   FRIEND_TEST(FutureRefTest, TailRemoved);
   FRIEND_TEST(FutureRefTest, HeadRemoved);
-};
+};  // namespace arrow

Review comment:
       ```suggestion
   };
   ```

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -581,4 +646,82 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+struct Continue {
+  template <typename T>
+  operator util::optional<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+util::optional<T> Break(T break_value = {}) {
+  return util::optional<T>{std::move(break_value)};
+}
+
+template <typename T = detail::Empty>
+using ControlFlow = util::optional<T>;
+
+/// \brief Loop through an asynchronous sequence
+///
+/// \param[in] iterate A generator of Future<ControlFlow<BreakValue>>. On completion of
+/// each yielded future the resulting ControlFlow will be examined. A Break will terminate
+/// the loop, while a Continue will re-invoke `iterate`. \return A future which will
+/// complete when a Future returned by iterate completes with a Break
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::value_type>
+Future<BreakValueType> Loop(Iterate iterate) {
+  auto break_fut = Future<BreakValueType>::Make();
+
+  struct Callback {
+    bool CheckForTermination(const Result<Control>& control_res) {
+      if (!control_res.ok()) {
+        break_fut.MarkFinished(control_res.status());
+        return true;
+      }
+      if (control_res->has_value()) {
+        break_fut.MarkFinished(*std::move(*control_res));
+        return true;
+      }
+      return false;
+    }
+
+    void operator()(const Result<Control>& maybe_control) && {
+      if (CheckForTermination(maybe_control)) return;
+
+      auto control_fut = iterate();
+      while (true) {
+        if (control_fut.is_finished()) {
+          // There's no need to AddCallback on a finished future; we can
+          // CheckForTermination now. This also avoids recursion and potential stack
+          // overflow.
+          if (CheckForTermination(control_fut.result())) return;
+
+          control_fut = iterate();
+        } else {
+          std::function<Callback()> callback_factory = [this]() { return *this; };
+          if (control_fut.TryAddCallback(callback_factory)) {
+            break;
+          }
+          // Else we tried to add a callback but someone had stolen in and marked the
+          // future finished so we can just resume iteration
+        }

Review comment:
       Nit: this loop could be simplified a bit further: https://github.com/westonpace/arrow/pull/4

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -792,65 +855,98 @@ class ThreadedTableReader : public BaseTableReader {
     ARROW_ASSIGN_OR_RAISE(auto istream_it,
                           io::MakeInputStreamIterator(input_, read_options_.block_size));
 
-    int32_t block_queue_size = thread_pool_->GetCapacity();
-    ARROW_ASSIGN_OR_RAISE(auto rh_it,
-                          MakeReadaheadIterator(std::move(istream_it), block_queue_size));
-    buffer_iterator_ = CSVBufferIterator::Make(std::move(rh_it));
+    // TODO: use io_executor_ here, see ARROW-11590
+    ARROW_ASSIGN_OR_RAISE(auto background_executor, internal::ThreadPool::Make(1));
+    ARROW_ASSIGN_OR_RAISE(auto bg_it, MakeBackgroundGenerator(std::move(istream_it),
+                                                              background_executor.get()));
+    AsyncGenerator<std::shared_ptr<Buffer>> wrapped_bg_it =
+        [bg_it, background_executor]() { return bg_it(); };
+
+    auto transferred_it =
+        MakeTransferredGenerator(std::move(wrapped_bg_it), cpu_executor_);
+
+    int32_t block_queue_size = cpu_executor_->GetCapacity();
+    auto rh_it = MakeReadaheadGenerator(std::move(transferred_it), block_queue_size);
+    buffer_generator_ = CSVBufferIterator::MakeAsync(std::move(rh_it));
     return Status::OK();
   }
 
-  Result<std::shared_ptr<Table>> Read() override {
-    task_group_ = internal::TaskGroup::MakeThreaded(thread_pool_);
+  Result<std::shared_ptr<Table>> Read() override { return ReadAsync().result(); }
+
+  Future<std::shared_ptr<Table>> ReadAsync() override {
+    task_group_ = internal::TaskGroup::MakeThreaded(cpu_executor_);
+
+    auto self = shared_from_this();
+    return ProcessFirstBuffer().Then([self](const std::shared_ptr<Buffer> first_buffer) {

Review comment:
       ```suggestion
       return ProcessFirstBuffer().Then([self](std::shared_ptr<Buffer> first_buffer) {
   ```




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r566305116



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue
+      for (int i = 0; i < max_readahead; i++) {
+        auto next = source_generator();
+        next.AddCallback(mark_finished_if_done);
+        readahead_queue.push(std::move(next));
+      }
+    }
+    // Pop one and add one
+    auto result = readahead_queue.front();
+    readahead_queue.pop();
+    if (*finished) {
+      readahead_queue.push(Future<T>::MakeFinished(IterationTraits<T>::End()));
+    } else {
+      auto back_of_queue = source_generator();
+      back_of_queue.AddCallback(mark_finished_if_done);
+      readahead_queue.push(std::move(back_of_queue));
+    }
+    return result;
+  };
+}
+
+/// \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.
+template <typename T, typename V>
+AsyncGenerator<V> TransformAsyncGenerator(AsyncGenerator<T> generator,
+                                          Transformer<T, V> transformer) {
+  return TransformingGenerator<T, V>(generator, transformer);
+}
+
+namespace detail {
+
+template <typename T>
+struct BackgroundIteratorPromise : ReadaheadPromise {
+  ~BackgroundIteratorPromise() override {}
+
+  explicit BackgroundIteratorPromise(Iterator<T>* it) : it_(it) {}
+
+  bool Call() override {
+    auto next = it_->Next();
+    auto finished = next == IterationTraits<T>::End();
+    out_.MarkFinished(std::move(next));
+    return finished;
+  }
+
+  void End() override { out_.MarkFinished(IterationTraits<T>::End()); }
+
+  Iterator<T>* it_;
+  Future<T> out_ = Future<T>::Make();
+};
+
+}  // namespace detail
+
+/// \brief Async generator that iterates on an underlying iterator in a
+/// separate thread.
+template <typename T>
+class BackgroundIterator {
+  using PromiseType = typename detail::BackgroundIteratorPromise<T>;
+
+ public:
+  explicit BackgroundIterator(Iterator<T> it, internal::Executor* executor)
+      : it_(new Iterator<T>(std::move(it))),
+        queue_(new detail::ReadaheadQueue(0)),
+        executor_(executor),
+        done_() {}
+
+  ~BackgroundIterator() {
+    if (queue_) {
+      // Make sure the queue doesn't call any promises after this object
+      // is destroyed.
+      queue_->EnsureShutdownOrDie();
+    }
+  }
+
+  ARROW_DEFAULT_MOVE_AND_ASSIGN(BackgroundIterator);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(BackgroundIterator);
+
+  Future<T> operator()() {
+    if (done_) {
+      return Future<T>::MakeFinished(IterationTraits<T>::End());
+    }
+    auto promise = std::unique_ptr<PromiseType>(new PromiseType{it_.get()});
+    auto result = Future<T>(promise->out_);
+    // TODO: Need a futuristic version of ARROW_RETURN_NOT_OK
+    auto append_status = queue_->Append(
+        static_cast<std::unique_ptr<detail::ReadaheadPromise>>(std::move(promise)));
+    if (!append_status.ok()) {
+      return Future<T>::MakeFinished(append_status);
+    }
+
+    result.AddCallback([this](const Result<T>& result) {
+      if (!result.ok() || result.ValueUnsafe() == IterationTraits<T>::End()) {
+        done_ = true;
+      }
+    });
+
+    return executor_->Transfer(result);

Review comment:
       Hmm... I'm not sure. Can we discuss this with Ben (e.g. on Zulip)? It would seem more logical to me if `Then` would by default schedule the callback on the current executor (that is, the executor of the thread calling `Then`). 




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r564948907



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {

Review comment:
       Wouldn't that callable still need to be copied?
   
   If they carry much state they are probably un-copyable and they wouldn't have been able to fit into a std::function in the first place.  There are many places where we copy these functions around.  For example, the visitor is immediately copied again in line 39.
   
   I've been operating under the assumption that anything put into a std::function should be cheap to copy.  In reader.cc you'll notice that both transformers are quite big and stateful so I go to the effort of wrapping them in a shared_ptr and capturing that shared_ptr with a lambda.
   
   Also, `VisitAsyncGenerator` is eventually going to expand to `VisitAsyncGenerator(AsyncGenerator<T> generator, std::function<Status(T)> visitor, int parallelism)`.  In that case the visitor is going to be copied up to `parallelism` times so it is important it be copyable.




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r572141313



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -556,6 +559,38 @@ inline bool WaitForAll(const std::vector<Future<T>*>& futures,
   return waiter->Wait(seconds);
 }
 
+/// \brief Create a Future which completes when all of `futures` complete.
+///
+/// The future's result is a vector of the results of `futures`.
+/// Note that this future will never be marked "failed"; failed results
+/// will be stored in the result vector alongside successful results.
+template <typename T>
+Future<std::vector<Result<T>>> All(std::vector<Future<T>> futures) {
+  struct State {
+    explicit State(std::vector<Future<T>> f)
+        : futures(std::move(f)), n_remaining(futures.size()) {}
+
+    std::vector<Future<T>> futures;
+    std::atomic<size_t> n_remaining;
+  };

Review comment:
       I was just wondering if perhaps `FutureWaiter` would allow a more efficient implementation. If it doesn't, then don't bother.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r555194653



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;

Review comment:
       done

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      util::optional<Result<V>> next = Pump();
+      if (next.has_value()) {
+        return *next;
+      }
+      ARROW_ASSIGN_OR_RAISE(last_value_, it_.Next());
+    }
+    return IterationTraits<V>::End();
+  }
+
+ private:
+  Iterator<T> it_;
+  Transformer<T, V> transformer_;
+  util::optional<T> last_value_;
+  bool finished_;

Review comment:
       done




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565972530



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;

Review comment:
       Done.

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {

Review comment:
       Done.




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r572167614



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,382 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;

Review comment:
       Should we make the file name (`async_iterator.h`) and the type name (`AsyncGenerator`) consistent? I don't know which convention is the most common.




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565169302



##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NTASKS = 100;
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+
+  std::vector<Future<std::shared_ptr<Table>>> task_futures(NTASKS);
+  for (int i = 0; i < NTASKS; i++) {
+    auto input = std::make_shared<io::BufferReader>(table_buffer);
+    ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+    task_futures[i] = reader->ReadAsync();
+  }
+  auto combined_future = All(task_futures);
+  combined_future.Wait();
+
+  ASSERT_OK_AND_ASSIGN(std::vector<Result<std::shared_ptr<Table>>> results,
+                       combined_future.result());
+  for (auto&& result : results) {
+    ASSERT_OK_AND_ASSIGN(auto table, result);
+    ASSERT_EQ(NROWS, table->num_rows());
+  }
+}
+
+void TestNestedParallelism(
+    std::shared_ptr<internal::ThreadPool> thread_pool,
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {
+  const int NROWS = 1000;
+  ASSERT_OK_AND_ASSIGN(auto table_buffer, MakeSampleCsvBuffer(NROWS));
+  auto input = std::make_shared<io::BufferReader>(table_buffer);
+  ASSERT_OK_AND_ASSIGN(auto reader, reader_factory(input));
+
+  Future<std::shared_ptr<Table>> table_future;
+
+  auto read_task = [&reader, &table_future]() mutable {
+    table_future = reader->ReadAsync();
+    return Status::OK();
+  };
+  ASSERT_OK_AND_ASSIGN(auto future, thread_pool->Submit(read_task));
+  ASSERT_TRUE(future.Wait(1));
+
+  if (future.is_finished()) {
+    ASSERT_TRUE(table_future.Wait(1));
+    if (table_future.is_finished()) {
+      ASSERT_OK_AND_ASSIGN(auto table, table_future.result());
+      ASSERT_EQ(table->num_rows(), NROWS);
+    }
+  }
+}  // namespace csv
+
+TEST(SerialReaderTests, Stress) {
+  auto task_factory = [](std::shared_ptr<io::InputStream> input_stream) {
+    return TableReader::Make(default_memory_pool(), io::AsyncContext(), input_stream,
+                             ReadOptions::Defaults(), ParseOptions::Defaults(),
+                             ConvertOptions::Defaults());
+  };
+  StressTableReader(task_factory);
+}
+
+TEST(SerialReaderTests, NestedParallelism) {
+  ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1));
+  auto task_factory = [](std::shared_ptr<io::InputStream> input_stream) {
+    return TableReader::Make(default_memory_pool(), io::AsyncContext(), input_stream,
+                             ReadOptions::Defaults(), ParseOptions::Defaults(),
+                             ConvertOptions::Defaults());
+  };
+  TestNestedParallelism(thread_pool, task_factory);
+}
+
+TEST(ThreadedReaderTests, Stress) {
+  ASSERT_OK_AND_ASSIGN(auto thread_pool, internal::ThreadPool::Make(1));
+  auto task_factory = [&thread_pool](std::shared_ptr<io::InputStream> input_stream)
+      -> Result<std::shared_ptr<TableReader>> {
+    ReadOptions read_options = ReadOptions::Defaults();
+    read_options.use_threads = true;
+    read_options.legacy_blocking_reads = true;
+    auto table_reader = TableReader::Make(
+        default_memory_pool(), io::AsyncContext(thread_pool.get()), input_stream,

Review comment:
       > However, it sounds like the existing definition is "This is the context on which the I/O operation should run". However, I think that should be owned by the FS.
   
   Hmm, you may be right that the FS and/or stream implementation should own it. Can you open a JIRA for that?
   
   > In my mind ctx is "This is the context on which work should resume when the I/O operation completes". 
   
   That would be a weird API IMHO. You generally pass function parameters for what gets done in the function, not what happens when the function finishes executing.
   




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565980516



##########
File path: cpp/src/arrow/util/task_group_test.cc
##########
@@ -292,5 +352,25 @@ TEST(ThreadedTaskGroup, StressFailingTaskGroupLifetime) {
       [&] { return TaskGroup::MakeThreaded(thread_pool.get()); });
 }
 
+TEST(ThreadedTaskGroup, FinishNotSticky) {
+  std::shared_ptr<ThreadPool> thread_pool;
+  ASSERT_OK_AND_ASSIGN(thread_pool, ThreadPool::Make(16));
+
+  TestFinishNotSticky([&] { return TaskGroup::MakeThreaded(thread_pool.get()); });
+}
+
+TEST(ThreadedTaskGroup, FinishNeverStarted) {
+  std::shared_ptr<ThreadPool> thread_pool;
+  ASSERT_OK_AND_ASSIGN(thread_pool, ThreadPool::Make(4));
+  TestFinishNeverStarted(TaskGroup::MakeThreaded(thread_pool.get()));
+}
+
+TEST(ThreadedTaskGroup, FinishAlreadyCompleted) {
+  std::shared_ptr<ThreadPool> thread_pool;
+  ASSERT_OK_AND_ASSIGN(thread_pool, ThreadPool::Make(16));
+
+  TestFinishAlreadyCompleted([&] { return TaskGroup::MakeThreaded(thread_pool.get()); });
+}

Review comment:
       Two of the three I could add.  The sticky case doesn't really work since it required `FinishAsync` to truly be async and not block.

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}
+
+TEST(TestAsyncIteratorTransform, SkipSome) {
+  auto original = AsyncVectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = TransformAsyncGenerator(std::move(original), filter);
+  AssertAsyncGeneratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestAsyncUtil, ReadaheadFailed) {
+  auto source = []() -> Future<TestInt> {
+    return Future<TestInt>::MakeFinished(Status::Invalid("X"));
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  auto next = readahead();
+  ASSERT_EQ(Status::Invalid("X"), next.status());
+}
+
+TEST(TestAsyncUtil, Readahead) {
+  int num_delivered = 0;
+  auto source = [&num_delivered]() {
+    if (num_delivered < 5) {
+      return Future<TestInt>::MakeFinished(num_delivered++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  // Should not pump until first item requested
+  ASSERT_EQ(0, num_delivered);
+
+  auto first = readahead();
+  // At this point the pumping should have happened
+  ASSERT_EQ(5, num_delivered);
+  ASSERT_EQ(0, first.result()->value);

Review comment:
       Done.

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());

Review comment:
       Done.

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -150,20 +154,21 @@ struct CSVBlock {
   std::function<Status(int64_t)> consume_bytes;
 };
 
+// This is an unfortunate side-effect of using optional<T> as the iterator in the
+// CSVBlock iterator.  We need to be able to compare with
+// IterationTraits<optional<T>>::End() and empty optionals will always compare true but
+// the optional copmarator won't compile if the underlying type isn't comparable

Review comment:
       Fixed.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565963530



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +187,122 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_ = false;
+  bool ready_for_next_ = false;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<Result<TransformFlow<V>>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  // Calls the transform function on the current value.  Can return in several ways
+  // * If the next value is requested (e.g. skip) it will return an empty optional
+  // * If an invalid status is encountered that will be returned
+  // * If finished it will return IterationTraits<V>::End()
+  // * If a value is returned by the transformer that will be returned
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      ARROW_ASSIGN_OR_RAISE(util::optional<V> next, Pump());
+      if (next.has_value()) {
+        return *next;
+      }
+      ARROW_ASSIGN_OR_RAISE(last_value_, it_.Next());
+    }
+    return IterationTraits<V>::End();
+  }
+
+ private:
+  Iterator<T> it_;
+  Transformer<T, V> transformer_;
+  util::optional<T> last_value_;
+  bool finished_ = false;
+};
+
+/// \brief Transforms an iterator according to a transformer, returning a new Iterator.
+///
+/// The transformer will be called on each element of the source iterator and for each
+/// call it can yield a value, skip, or finish the iteration.  When yielding a value the
+/// transformer can choose to consume the source item (the default, ready_for_next = true)
+/// or to keep it and it will be called again on the same value.

Review comment:
       It would be possible to create an async generator that operated in this way.  However, I'm not able to come up with a "transform" style factory that wraps a single function and allows that function to be similarly transformed by synchronous iterators.
   
   Since the only iterators I truly need to be "transform functions" are the CSV block reader and chunker (to avoid duplication) I could drop the ready_for_next (this also prevents the resulting generator from being async re-entrant) and decompressor style generators would have to use some other mechanism for creation (and I just wouldn't create a synchronous counterpart).

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +187,122 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_ = false;
+  bool ready_for_next_ = false;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<Result<TransformFlow<V>>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  // Calls the transform function on the current value.  Can return in several ways
+  // * If the next value is requested (e.g. skip) it will return an empty optional
+  // * If an invalid status is encountered that will be returned
+  // * If finished it will return IterationTraits<V>::End()
+  // * If a value is returned by the transformer that will be returned
+  Result<util::optional<V>> Pump() {

Review comment:
       Not anymore.




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565929811



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      util::optional<Result<V>> next = Pump();
+      if (next.has_value()) {
+        return *next;

Review comment:
       It shouldn't be necessary, but unfortunately some compilers are finicky.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565967203



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *one.result());
+  }
+
+  ASSERT_TRUE(two.Wait(0.5));
+  ASSERT_TRUE(two.is_finished());
+  if (two.is_finished()) {
+    ASSERT_EQ(IterationTraits<TestInt>::End(), *two.result());
+  }
+}
+
+TEST(TestAsyncUtil, SynchronousFinish) {
+  AsyncGenerator<TestInt> generator = []() {
+    return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+  };
+  Transformer<TestInt, TestInt> skip_all = [](TestInt value) { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, skip_all);
+  auto future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_OK_AND_ASSIGN(auto actual, future.result());
+  ASSERT_EQ(std::vector<TestInt>(), actual);
+}
+
+TEST(TestAsyncUtil, CompleteBackgroundStressTest) {
+  auto expected = RangeVector(100);
+  std::vector<Future<std::vector<TestInt>>> futures;
+  for (unsigned int i = 0; i < 100; i++) {
+    auto background = BackgroundAsyncVectorIt(expected);
+    futures.push_back(CollectAsyncGenerator(background));
+  }
+  auto combined = All(futures);
+  combined.Wait(2);
+  if (combined.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto completed_vectors, combined.result());
+    for (auto&& vector : completed_vectors) {
+      ASSERT_EQ(vector, expected);
+    }
+  } else {
+    FAIL() << "After 2 seconds all background iterators had not finished collecting";
+  }
+}
+
+TEST(TestAsyncUtil, StackOverflow) {
+  int counter = 0;
+  AsyncGenerator<TestInt> generator = [&counter]() {
+    if (counter < 1000000) {
+      return Future<TestInt>::MakeFinished(counter++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  Transformer<TestInt, TestInt> discard =
+      [](TestInt next) -> Result<TransformFlow<TestInt>> { return TransformSkip(); };
+  auto transformed = TransformAsyncGenerator(generator, discard);
+  auto collected_future = CollectAsyncGenerator(transformed);
+  ASSERT_TRUE(collected_future.Wait(5));
+  if (collected_future.is_finished()) {
+    ASSERT_OK_AND_ASSIGN(auto collected, collected_future.result());
+    ASSERT_EQ(0, collected.size());
+  }
+}
+
+TEST(TestAsyncUtil, Visit) {
+  auto generator = AsyncVectorIt({1, 2, 3});
+  unsigned int sum = 0;
+  auto sum_future = VisitAsyncGenerator<TestInt>(generator, [&sum](TestInt item) {
+    sum += item.value;
+    return Status::OK();
+  });
+  // Should be superfluous
+  sum_future.Wait();
+  ASSERT_EQ(6, sum);
+}
+
+TEST(TestAsyncUtil, Collect) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto generator = AsyncVectorIt(expected);
+  auto collected = CollectAsyncGenerator(generator);
+  ASSERT_EQ(expected, *collected.result());
+}
+
+template <typename T>
+Transformer<T, T> MakeRepeatN(int repeat_count) {
+  int current_repeat = 0;
+  return [repeat_count, current_repeat](T next) mutable -> Result<TransformFlow<T>> {
+    current_repeat++;
+    bool ready_for_next = false;
+    if (current_repeat == repeat_count) {
+      current_repeat = 0;
+      ready_for_next = true;
+    }
+    return TransformYield(next, ready_for_next);
+  };
+}
+
+TEST(TestIteratorTransform, Repeating) {
+  auto original = VectorIt({1, 2, 3});
+  auto repeated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                            MakeRepeatN<TestInt>(2));
+  AssertIteratorMatch({1, 1, 2, 2, 3, 3}, std::move(repeated));
+}
+
+template <typename T>
+Transformer<T, T> MakeFilter(std::function<bool(T&)> filter) {
+  return [filter](T next) -> Result<TransformFlow<T>> {
+    if (filter(next)) {
+      return TransformYield(next);
+    } else {
+      return TransformSkip();
+    }
+  };
+}
+
+template <typename T>
+Transformer<T, T> MakeAbortOnSecond() {
+  int counter = 0;
+  return [counter](T next) mutable -> Result<TransformFlow<T>> {
+    if (counter++ == 1) {
+      return Status::Invalid("X");
+    }
+    return TransformYield(next);
+  };
+}
+
+TEST(TestIteratorTransform, SkipSome) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, SkipAll) {
+  // Exercises TransformSkip
+  auto original = VectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return false; });
+  auto filtered = MakeTransformedIterator(std::move(original), filter);
+  AssertIteratorMatch({}, std::move(filtered));
+}
+
+TEST(TestIteratorTransform, Abort) {
+  auto original = VectorIt({1, 2, 3});
+  auto transformed =
+      MakeTransformedIterator(std::move(original), MakeAbortOnSecond<TestInt>());
+  ASSERT_OK(transformed.Next());
+  ASSERT_RAISES(Invalid, transformed.Next());
+}
+
+TEST(TestAsyncIteratorTransform, SkipSome) {
+  auto original = AsyncVectorIt({1, 2, 3});
+  auto filter = MakeFilter<TestInt>([](TestInt& t) { return t.value != 2; });
+  auto filtered = TransformAsyncGenerator(std::move(original), filter);
+  AssertAsyncGeneratorMatch({1, 3}, std::move(filtered));
+}
+
+TEST(TestAsyncUtil, ReadaheadFailed) {
+  auto source = []() -> Future<TestInt> {
+    return Future<TestInt>::MakeFinished(Status::Invalid("X"));
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  auto next = readahead();
+  ASSERT_EQ(Status::Invalid("X"), next.status());
+}
+
+TEST(TestAsyncUtil, Readahead) {
+  int num_delivered = 0;
+  auto source = [&num_delivered]() {
+    if (num_delivered < 5) {
+      return Future<TestInt>::MakeFinished(num_delivered++);
+    } else {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+  };
+  auto readahead = AddReadahead<TestInt>(source, 10);
+  // Should not pump until first item requested

Review comment:
       I'm not sure that it is less useful for an async reader.  It isn't a matter of blocking either.  The threaded table reader could have the same behavior.  Also, the serial table reader was already different from the threaded table reader in this regard (the async reader mimics the serial reader)...
   
   ```
     ARROW_ASSIGN_OR_RAISE(auto csv_reader, arrow::csv::TableReader::Make(...);  // Should the read start here?
     ARROW_ASSIGN_OR_RAISE(auto table, csv_reader->Read());                                 // Or here?
   ```
   As a user I really wouldn't expect the file to start getting read until I called `Read()`.  I agree that it shouldn't have much impact though and I'd be content to change it.

##########
File path: cpp/src/arrow/util/task_group.h
##########
@@ -63,6 +63,20 @@ class ARROW_EXPORT TaskGroup : public std::enable_shared_from_this<TaskGroup> {
   /// task (or subgroup).
   virtual Status Finish() = 0;
 
+  /// Returns a future that will complete the first time all tasks are finished.
+  /// This should be called only after all top level tasks
+  /// have been added to the task group.
+  ///
+  /// If you are using a TaskGroup asyncrhonously there are a few considerations to keep

Review comment:
       Fixed

##########
File path: cpp/src/arrow/util/task_group.cc
##########
@@ -135,6 +149,18 @@ class ThreadedTaskGroup : public TaskGroup {
       // before cv.notify_one() has returned
       std::unique_lock<std::mutex> lock(mutex_);
       cv_.notify_one();
+      if (completion_future_.has_value()) {
+        // MarkFinished could be slow.  We don't want to call it while we are holding
+        // the lock.
+        // TODO: If optional is thread safe then we can skip this locking entirely

Review comment:
       Dropped the TODO




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565963804



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -428,10 +547,16 @@ struct ReadaheadIteratorPromise : ReadaheadPromise {
 
   explicit ReadaheadIteratorPromise(Iterator<T>* it) : it_(it) {}
 
-  void Call() override {
+  bool Call() override {
     assert(!called_);
     out_ = it_->Next();
     called_ = true;
+    return out_ == IterationTraits<T>::End();
+  }
+
+  void End() override {
+    // No need to do anything for the synchronous case.  No one is waiting on this
+    // called_ = true;

Review comment:
       Holdover from debugging.  I have uncommented this line.

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -129,11 +130,44 @@ template <typename T>
 inline Iterator<T> EmptyIt() {
   return MakeEmptyIterator<T>();
 }
-
 inline Iterator<TestInt> VectorIt(std::vector<TestInt> v) {
   return MakeVectorIterator<TestInt>(std::move(v));
 }
 
+std::function<Future<TestInt>()> AsyncVectorIt(std::vector<TestInt> v) {

Review comment:
       Done.

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -214,6 +255,270 @@ TEST(TestVectorIterator, RangeForLoop) {
   ASSERT_EQ(ints_it, ints.end());
 }
 
+template <typename T>
+Transformer<T, T> MakeFirstN(int n) {
+  int remaining = n;
+  return [remaining](T next) mutable -> Result<TransformFlow<T>> {
+    if (remaining > 0) {
+      remaining--;
+      return TransformYield(next);
+    }
+    return TransformFinish();
+  };
+}
+
+TEST(TestIteratorTransform, Truncating) {
+  auto original = VectorIt({1, 2, 3});
+  auto truncated = MakeTransformedIterator(std::move(original), MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1, 2}, std::move(truncated));
+}
+
+TEST(TestIteratorTransform, TestPointer) {
+  auto original = VectorIt<std::shared_ptr<int>>(
+      {std::make_shared<int>(1), std::make_shared<int>(2), std::make_shared<int>(3)});
+  auto truncated =
+      MakeTransformedIterator(std::move(original), MakeFirstN<std::shared_ptr<int>>(2));
+  ASSERT_OK_AND_ASSIGN(auto result, truncated.ToVector());
+  ASSERT_EQ(2, result.size());
+}
+
+TEST(TestIteratorTransform, TruncatingShort) {
+  // Tests the failsafe case where we never call Finish
+  auto original = VectorIt({1});
+  auto truncated = MakeTransformedIterator<TestInt, TestInt>(std::move(original),
+                                                             MakeFirstN<TestInt>(2));
+  AssertIteratorMatch({1}, std::move(truncated));
+}
+
+TEST(TestAsyncUtil, Background) {
+  std::vector<TestInt> expected = {1, 2, 3};
+  auto background = BackgroundAsyncVectorIt(expected);
+  auto future = CollectAsyncGenerator(background);
+  ASSERT_FALSE(future.is_finished());
+  future.Wait();
+  ASSERT_TRUE(future.is_finished());
+  ASSERT_EQ(expected, *future.result());
+}
+
+struct SlowEmptyIterator {
+  Result<TestInt> Next() {
+    if (called_) {
+      return Status::Invalid("Should not have been called twice");
+    }
+    SleepFor(0.1);
+    return IterationTraits<TestInt>::End();
+  }
+
+ private:
+  bool called_ = false;
+};
+
+TEST(TestAsyncUtil, BackgroundRepeatEnd) {
+  // Ensure that the background iterator properly fulfills the asyncgenerator contract
+  // and can be called after it ends.
+  auto iterator = Iterator<TestInt>(SlowEmptyIterator());
+  ASSERT_OK_AND_ASSIGN(
+      auto background_iter,
+      MakeBackgroundIterator(std::move(iterator), internal::GetCpuThreadPool()));
+
+  auto one = background_iter();
+  auto two = background_iter();
+
+  ASSERT_TRUE(one.Wait(0.5));
+
+  if (one.is_finished()) {

Review comment:
       Changed to ASSERT_FINISHES...




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565173337



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -556,6 +559,38 @@ inline bool WaitForAll(const std::vector<Future<T>*>& futures,
   return waiter->Wait(seconds);
 }
 
+/// \brief Create a Future which completes when all of `futures` complete.
+///
+/// The future's result is a vector of the results of `futures`.
+/// Note that this future will never be marked "failed"; failed results
+/// will be stored in the result vector alongside successful results.
+template <typename T>
+Future<std::vector<Result<T>>> All(std::vector<Future<T>> futures) {
+  struct State {
+    explicit State(std::vector<Future<T>> f)
+        : futures(std::move(f)), n_remaining(futures.size()) {}
+
+    std::vector<Future<T>> futures;
+    std::atomic<size_t> n_remaining;
+  };

Review comment:
       Yes, I meant that perhaps it would be easy to add a non-blocking facility to `FutureWaiter` internals. It is designed to receive "done" notifications from futures after all (though perhaps it's not more efficient than `AddCallback`?).




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



[GitHub] [arrow] pitrou commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565177180



##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -16,6 +16,7 @@
 // under the License.
 
 #include "arrow/util/iterator.h"
+#include "arrow/util/async_iterator.h"

Review comment:
       Ah, if clang does it then nevermind.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565953149



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {

Review comment:
       I've moved it out of public.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565936241



##########
File path: cpp/src/arrow/csv/reader_test.cc
##########
@@ -0,0 +1,169 @@
+// 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 <cstdint>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/future.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+void StressTableReader(
+    std::function<Result<std::shared_ptr<TableReader>>(std::shared_ptr<io::InputStream>)>
+        reader_factory) {

Review comment:
       Done.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565542380



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {

Review comment:
       > (I'd also mention that std::function makes backtraces in debug mode rather annoying, but that's a much smaller concern)
   I can live with the backtraces (but I agree they are annoying) but it also makes it pretty much impossible to "step into" as well.
   
   Are you sure the overhead is std::function vs function pointer/lambda?  I wonder how much optimization the compiler can really do with an arbitrary callable?
   
   For example, I've been bitten by lambda before when I captured an accumulator variable by reference because the accumulator, which had previously been a register, now required a full RAM write for every increment (since other threads could now conceivably be observing this).




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565971907



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();

Review comment:
       Done.

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {

Review comment:
       Changed to a move.

##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {

Review comment:
       Changed to a move.  The generator is no longer copied.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565958445



##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +615,74 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>
+Future<BreakValueType> Loop(Iterate iterate) {
+  auto break_fut = Future<BreakValueType>::Make();
+
+  struct Callback {
+    bool CheckForTermination(const Result<Control>& maybe_control) {
+      if (!maybe_control.ok() || maybe_control->IsBreak()) {
+        Result<BreakValueType> maybe_break = maybe_control.Map(Control::MoveBreakValue);
+        break_fut.MarkFinished(std::move(maybe_break));
+        return true;
+      }
+      return false;
+    }
+
+    void operator()(const Result<Control>& maybe_control) && {
+      if (CheckForTermination(maybe_control)) return;
+
+      auto control_fut = iterate();
+      while (control_fut.is_finished()) {
+        // There's no need to AddCallback on a finished future; we can CheckForTermination
+        // now. This also avoids recursion and potential stack overflow.

Review comment:
       I struggled to create a test case that could reproduce this.  It seems like it should be reproducible but even adding a slow callback (so MarkFinished took a long time) it was difficult to get a task to be added and then finish just when I wanted it to.  I have added some logic however so that this race condition should not be possible any longer.




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



[GitHub] [arrow] bkietz commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r556129806



##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,6 +832,145 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(3, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(Status::IOError("XYZ"), results[1].status());
+    ASSERT_EQ(3, *results[2]);
+  }));
+
+  f1.MarkFinished(1);
+  f2.MarkFinished(Status::IOError("XYZ"));
+  f3.MarkFinished(3);
+
+  AssertFinished(combined);
+}
+
+TEST(FutureLoopTest, Sync) {
+  struct {
+    int i = 0;
+    Future<int> Get() { return Future<int>::MakeFinished(i++); }
+  } IntSource;
+
+  bool do_fail = false;
+  std::vector<int> ints;
+  auto loop_body = [&] {
+    return IntSource.Get().Then([&](int i) -> Result<ControlFlow<int>> {
+      if (do_fail && i == 3) {
+        return Status::IOError("xxx");
+      }
+
+      if (i == 5) {
+        int sum = 0;
+        for (int i : ints) sum += i;
+        return Break(sum);
+      }
+
+      ints.push_back(i);
+      return Continue();
+    });
+  };
+
+  {
+    auto sum_fut = Loop(loop_body);
+    AssertSuccessful(sum_fut);
+
+    ASSERT_OK_AND_ASSIGN(auto sum, sum_fut.result());
+    ASSERT_EQ(sum, 0 + 1 + 2 + 3 + 4);
+  }
+
+  {
+    do_fail = true;
+    IntSource.i = 0;
+    auto sum_fut = Loop(loop_body);
+    AssertFailed(sum_fut);
+    ASSERT_RAISES(IOError, sum_fut.result());
+  }
+}
+
+TEST(FutureLoopTest, EmptyBreakValue) {
+  Future<> none_fut =
+      Loop([&] { return Future<>::MakeFinished().Then([&](...) { return Break(); }); });
+  AssertSuccessful(none_fut);
+}
+
+TEST(FutureLoopTest, MoveOnlyBreakValue) {
+  Future<MoveOnlyDataType> one_fut = Loop([&] {
+    return Future<int>::MakeFinished(1).Then(
+        [&](int i) { return Break(MoveOnlyDataType(i)); });
+  });
+  AssertSuccessful(one_fut);
+  ASSERT_OK_AND_ASSIGN(auto one, std::move(one_fut).result());
+  ASSERT_EQ(one, 1);
+}
+
+TEST(FutureLoopTest, StackOverflow) {
+  // Looping over futures is normally a rather recursive task.  If the futures complete
+  // synchronously (because they are already finished) it could lead to a stack overflow
+  // if care is not taken.
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 1000000) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());

Review comment:
       For lambdas without an explicit trailing return type, all return statements *must* return the same type. 




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r572267087



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,382 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;

Review comment:
       I have gone back and forth a little (hence this discrepancy). I don't think asynchronous iterators are terribly common in general.  When they are used (e.g. javascript) it is in conjunction with syntactic "await" sugar to create something that can actually be iterated (e.g. `for await...of`).  The only way to truly "iterate" these chains is to use the Collect/Visit utilities (or the underlying Loop) which I feel isn't quite the same thing.  So asynchronous iterator isn't an ideal name.
   
   Generator is used in python for creating an iterator with yield statements which is an entirely separate concept.  It's used exactly the same in javascript.  In fact, in javascript there is an "asynchronous generator" which allows you to create asynchronous functions that yield (which is not what we are doing there).  So the asynchronous generator name isn't perfect either.
   
   In the FRP world these might be called events and event streams but I think that fits a push-based model better.
   
   I think, in the interest of "perfect is the enemy of good" I will stick with AsyncGenerator everywhere and rename `async_iterator` to `async_generator`.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565973039



##########
File path: cpp/src/arrow/util/async_iterator.h
##########
@@ -0,0 +1,295 @@
+// 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.
+
+#pragma once
+#include <queue>
+
+#include "arrow/util/functional.h"
+#include "arrow/util/future.h"
+#include "arrow/util/iterator.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+
+template <typename T>
+using AsyncGenerator = std::function<Future<T>()>;
+
+/// Iterates through a generator of futures, visiting the result of each one and
+/// returning a future that completes when all have been visited
+template <typename T>
+Future<> VisitAsyncGenerator(AsyncGenerator<T> generator,
+                             std::function<Status(T)> visitor) {
+  auto loop_body = [generator, visitor] {
+    auto next = generator();
+    return next.Then([visitor](const T& result) -> Result<ControlFlow<detail::Empty>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(detail::Empty());
+      } else {
+        auto visited = visitor(result);
+        if (visited.ok()) {
+          return Continue();
+        } else {
+          return visited;
+        }
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T>
+Future<std::vector<T>> CollectAsyncGenerator(AsyncGenerator<T> generator) {
+  auto vec = std::make_shared<std::vector<T>>();
+  auto loop_body = [generator, vec] {
+    auto next = generator();
+    return next.Then([vec](const T& result) -> Result<ControlFlow<std::vector<T>>> {
+      if (result == IterationTraits<T>::End()) {
+        return Break(*vec);
+      } else {
+        vec->push_back(result);
+        return Continue();
+      }
+    });
+  };
+  return Loop(loop_body);
+}
+
+template <typename T, typename V>
+class TransformingGenerator {
+ public:
+  explicit TransformingGenerator(AsyncGenerator<T> generator,
+                                 Transformer<T, V> transformer)
+      : finished_(), last_value_(), generator_(generator), transformer_(transformer) {}
+
+  // See comment on TransformingIterator::Pump
+  Result<util::optional<V>> Pump() {
+    if (!finished_ && last_value_.has_value()) {
+      ARROW_ASSIGN_OR_RAISE(TransformFlow<V> next, transformer_(*last_value_));
+      if (next.ReadyForNext()) {
+        if (*last_value_ == IterationTraits<T>::End()) {
+          finished_ = true;
+        }
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::nullopt;
+  }
+
+  Future<V> operator()() {
+    while (true) {
+      auto maybe_next_result = Pump();
+      if (!maybe_next_result.ok()) {
+        return Future<V>::MakeFinished(maybe_next_result.status());
+      }
+      auto maybe_next = maybe_next_result.ValueUnsafe();
+      if (maybe_next.has_value()) {
+        return Future<V>::MakeFinished(*maybe_next);
+      }
+
+      auto next_fut = generator_();
+      // If finished already, process results immediately inside the loop to avoid stack
+      // overflow
+      if (next_fut.is_finished()) {
+        auto next_result = next_fut.result();
+        if (next_result.ok()) {
+          last_value_ = *next_result;
+        } else {
+          return Future<V>::MakeFinished(next_result.status());
+        }
+        // Otherwise, if not finished immediately, add callback to process results
+      } else {
+        return next_fut.Then([this](const Result<T>& next_result) {
+          if (next_result.ok()) {
+            last_value_ = *next_result;
+            return (*this)();
+          } else {
+            return Future<V>::MakeFinished(next_result.status());
+          }
+        });
+      }
+    }
+  }
+
+ protected:
+  bool finished_;
+  util::optional<T> last_value_;
+  AsyncGenerator<T> generator_;
+  Transformer<T, V> transformer_;
+};
+
+template <typename T>
+static std::function<void(const Result<T>&)> MakeCallback(
+    std::shared_ptr<bool> finished) {
+  return [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+}
+
+template <typename T>
+AsyncGenerator<T> AddReadahead(AsyncGenerator<T> source_generator, int max_readahead) {
+  // Using a shared_ptr instead of a lambda capture here because it's possible that
+  // the inner mark_finished_if_done outlives the outer lambda
+  auto finished = std::make_shared<bool>(false);
+  auto mark_finished_if_done = [finished](const Result<T>& next_result) {
+    if (!next_result.ok()) {
+      *finished = true;
+    } else {
+      auto next = *next_result;
+      *finished = (next == IterationTraits<T>::End());
+    }
+  };
+
+  std::queue<Future<T>> readahead_queue;
+  return [=]() mutable -> Future<T> {
+    if (readahead_queue.empty()) {
+      // This is the first request, let's pump the underlying queue

Review comment:
       See earlier comment.  Right now we are mimicking SerialTableReader which sets up the structures in the `Make` and starts the read in the `Read`.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565932779



##########
File path: cpp/src/arrow/csv/reader_benchmark.cc
##########
@@ -0,0 +1,139 @@
+// 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 <string>
+
+#include "arrow/buffer.h"
+#include "arrow/csv/options.h"
+#include "arrow/csv/reader.h"
+#include "arrow/csv/test_common.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace csv {
+
+class SlowInputStream : public io::InputStream {
+ public:
+  explicit SlowInputStream(std::shared_ptr<io::BufferReader> target, int64_t latency_ms)
+      : target_(std::move(target)) {
+    latency_s_ = static_cast<double>(latency_ms) / 1000.0;
+  }
+  virtual ~SlowInputStream() {}
+
+  Result<util::string_view> Peek(int64_t nbytes) override {
+    return target_->Peek(nbytes);
+  }
+  bool supports_zero_copy() const override { return target_->supports_zero_copy(); }
+  Status Close() override { return target_->Close(); }
+  Status Abort() override { return target_->Abort(); }
+  Result<int64_t> Tell() const override { return target_->Tell(); }
+  bool closed() const override { return target_->closed(); }
+  Result<int64_t> Read(int64_t nbytes, void* out) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes, out);
+  }
+  Result<std::shared_ptr<Buffer>> Read(int64_t nbytes) override {
+    if (latency_s_ > 0) {
+      SleepFor(latency_s_);
+    }
+    return target_->Read(nbytes);
+  }
+  Status Seek(int64_t pos) { return target_->Seek(pos); }
+
+ private:
+  std::shared_ptr<io::BufferReader> target_;
+  double latency_s_;
+};
+
+static ReadOptions CreateReadOptions(bool use_threads, bool use_async) {
+  auto result = csv::ReadOptions::Defaults();
+  result.use_threads = use_threads;
+  result.legacy_blocking_reads = !use_async;
+  // Simulate larger files by using smaller block files so the impact of multiple
+  // blocks is seen but we don't have to spend the time waiting on the large I/O
+  result.block_size = (1 << 20) / 100;
+  return result;
+}
+
+static std::shared_ptr<SlowInputStream> CreateStreamReader(std::shared_ptr<Buffer> buffer,
+                                                           int64_t latency_ms) {
+  auto buffer_reader = std::make_shared<io::BufferReader>(buffer);
+  return std::make_shared<SlowInputStream>(buffer_reader, latency_ms);
+}
+
+static void BenchmarkReader(benchmark::State& state, bool use_threads, bool use_async) {
+  auto latency_ms = state.range(0);
+  auto num_rows = state.range(1);
+  auto num_files = state.range(2);
+  if (num_files > 5 && use_threads && !use_async) {
+    state.SkipWithError("Would deadlock");
+  }
+  auto input_buffer = *MakeSampleCsvBuffer(num_rows);
+  // Hard coding # of threads so we don't deadlock if there are too few cores
+  ASSIGN_OR_ABORT(auto thread_pool, internal::ThreadPool::Make(6));
+  io::AsyncContext async_context(thread_pool.get());
+  while (state.KeepRunning()) {
+    std::vector<Future<std::shared_ptr<Table>>> table_futures;
+    for (int i = 0; i < num_files; i++) {
+      auto stream_reader = CreateStreamReader(input_buffer, latency_ms);
+      auto table_reader = *csv::TableReader::Make(
+          default_memory_pool(), async_context, stream_reader,
+          CreateReadOptions(use_threads, use_async), csv::ParseOptions::Defaults(),
+          csv::ConvertOptions::Defaults());
+      if (use_async) {
+        table_futures.push_back(table_reader->ReadAsync());
+      } else {
+        ASSERT_OK_AND_ASSIGN(auto table_future,
+                             async_context.executor->Submit(
+                                 [table_reader] { return table_reader->Read(); }));
+        table_futures.push_back(table_future);
+      }
+    }
+    auto combined = All(table_futures);
+    ASSIGN_OR_ABORT(auto result, combined.result());
+    for (auto&& table : result) {
+      ABORT_NOT_OK(table);
+    }
+  }
+  state.SetItemsProcessed(state.iterations() * num_rows);

Review comment:
       The benchmark has been removed.




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



[GitHub] [arrow] westonpace commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r565935517



##########
File path: cpp/src/arrow/csv/test_common.cc
##########
@@ -61,5 +61,47 @@ void MakeColumnParser(std::vector<std::string> items, std::shared_ptr<BlockParse
   ASSERT_EQ((*out)->num_rows(), items.size());
 }
 
+const std::vector<std::string> int64_rows = {"123", "4", "-317005557", "", "N/A", "0"};

Review comment:
       I moved these into an anonymous namespace.




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



[GitHub] [arrow] github-actions[bot] commented on pull request #9095: Feature/arrow 10183 2

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#issuecomment-754137607


   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   Could you open an issue for this pull request on JIRA?
   https://issues.apache.org/jira/browse/ARROW
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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



[GitHub] [arrow] bkietz commented on a change in pull request #9095: ARROW-10183: [C++] Apply composable futures to CSV

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #9095:
URL: https://github.com/apache/arrow/pull/9095#discussion_r554155616



##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -35,6 +36,13 @@
 
 namespace arrow {
 
+namespace detail {
+
+template <typename Signature>
+using result_of_t = typename std::result_of<Signature>::type;

Review comment:
       It's technically legal to have this both here and in future.h, but for clarity we should probably have a single alias decl. util/functional.h would be a reasonable place to put it

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      util::optional<Result<V>> next = Pump();
+      if (next.has_value()) {
+        return *next;
+      }
+      ARROW_ASSIGN_OR_RAISE(last_value_, it_.Next());
+    }
+    return IterationTraits<V>::End();
+  }
+
+ private:
+  Iterator<T> it_;
+  Transformer<T, V> transformer_;
+  util::optional<T> last_value_;
+  bool finished_;

Review comment:
       ```suggestion
     bool finished_ = false;
   ```

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {

Review comment:
       This loop is very confusing. Could you rewrite it with a singular condition (`while (!finished_)`, maybe) then include a break statement below?

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();
+  }
+
+  Result<V> Next() {
+    while (!finished_) {
+      util::optional<Result<V>> next = Pump();
+      if (next.has_value()) {
+        return *next;

Review comment:
       ```suggestion
           return std::move(*next);
   ```

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +611,73 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,
+          typename Control = typename detail::result_of_t<Iterate()>::ValueType,
+          typename BreakValueType = typename Control::BreakValueType>
+Future<BreakValueType> Loop(Iterate iterate) {

Review comment:
       Maybe the typedef for AsyncGenerator would be useful here; then we could write
   
   ```suggestion
   template <typename T>
   using AsyncGenerator = std::function<Future<T>()>;
   
   template <typename BreakValue>
   Future<BreakValue> Loop(AsyncGenerator<ControlFlow<BreakValue>> iterate) {
   ```

##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,6 +832,145 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(3, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(Status::IOError("XYZ"), results[1].status());
+    ASSERT_EQ(3, *results[2]);
+  }));
+
+  f1.MarkFinished(1);
+  f2.MarkFinished(Status::IOError("XYZ"));
+  f3.MarkFinished(3);
+
+  AssertFinished(combined);
+}
+
+TEST(FutureLoopTest, Sync) {
+  struct {
+    int i = 0;
+    Future<int> Get() { return Future<int>::MakeFinished(i++); }
+  } IntSource;
+
+  bool do_fail = false;
+  std::vector<int> ints;
+  auto loop_body = [&] {
+    return IntSource.Get().Then([&](int i) -> Result<ControlFlow<int>> {
+      if (do_fail && i == 3) {
+        return Status::IOError("xxx");
+      }
+
+      if (i == 5) {
+        int sum = 0;
+        for (int i : ints) sum += i;
+        return Break(sum);
+      }
+
+      ints.push_back(i);
+      return Continue();
+    });
+  };
+
+  {
+    auto sum_fut = Loop(loop_body);
+    AssertSuccessful(sum_fut);
+
+    ASSERT_OK_AND_ASSIGN(auto sum, sum_fut.result());
+    ASSERT_EQ(sum, 0 + 1 + 2 + 3 + 4);
+  }
+
+  {
+    do_fail = true;
+    IntSource.i = 0;
+    auto sum_fut = Loop(loop_body);
+    AssertFailed(sum_fut);
+    ASSERT_RAISES(IOError, sum_fut.result());
+  }
+}
+
+TEST(FutureLoopTest, EmptyBreakValue) {
+  Future<> none_fut =
+      Loop([&] { return Future<>::MakeFinished().Then([&](...) { return Break(); }); });
+  AssertSuccessful(none_fut);
+}
+
+TEST(FutureLoopTest, MoveOnlyBreakValue) {

Review comment:
       It'd be good to have a stress test for `Loop` as well

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {

Review comment:
       Could you also include a comment describing the control flow, the contract of transformer functions, ...?

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}

Review comment:
       Instead of having TransformFlow potentially be an error, perhaps we could just rely on Result and have `using Transformer = std::function<Result<TransformFlow<V>>(T)>;`?

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {

Review comment:
       I'd usually expect `Result<>` to be outermost in a return type:
   ```suggestion
     Result<util::optional<V>> Pump() {
   ```
   Additionally, that'd enable you to use `ARROW_RETURN_NOT_OK` below

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;

Review comment:
       Use member initializers for fields with no default constructor:
   ```suggestion
     bool finished_ = false;
     bool ready_for_next_ = false;
   ```

##########
File path: cpp/src/arrow/util/iterator.h
##########
@@ -186,6 +194,109 @@ class Iterator : public util::EqualityComparable<Iterator<T>> {
   Result<T> (*next_)(void*) = NULLPTR;
 };
 
+template <typename T>
+struct TransformFlow {
+  using YieldValueType = T;
+
+  TransformFlow(YieldValueType value, bool ready_for_next)
+      : finished_(false),
+        ready_for_next_(ready_for_next),
+        status_(),
+        yield_value_(std::move(value)) {}
+  TransformFlow(bool finished, bool ready_for_next)
+      : finished_(finished), ready_for_next_(ready_for_next), status_(), yield_value_() {}
+  TransformFlow(Status s)  // NOLINT runtime/explicit
+      : finished_(true), ready_for_next_(false), status_(s), yield_value_() {}
+
+  bool HasValue() const { return yield_value_.has_value(); }
+  bool Finished() const { return finished_; }
+  Status status() const { return status_; }
+  bool Ok() const { return status_.ok(); }
+  bool ReadyForNext() const { return ready_for_next_; }
+  T Value() const { return *yield_value_; }
+
+  bool finished_;
+  bool ready_for_next_;
+  Status status_;
+  util::optional<YieldValueType> yield_value_;
+};
+
+struct TransformFinish {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(true, true);
+  }
+};
+
+struct TransformSkip {
+  template <typename T>
+  operator TransformFlow<T>() && {  // NOLINT explicit
+    return TransformFlow<T>(false, true);
+  }
+};
+
+template <typename T>
+TransformFlow<T> TransformYield(T value = {}, bool ready_for_next = true) {
+  return TransformFlow<T>(std::move(value), ready_for_next);
+}
+
+template <typename T, typename V>
+using Transformer = std::function<TransformFlow<V>(T)>;
+
+template <typename T, typename V>
+class TransformIterator {
+ public:
+  explicit TransformIterator(Iterator<T> it, Transformer<T, V> transformer)
+      : it_(std::move(it)),
+        transformer_(std::move(transformer)),
+        last_value_(),
+        finished_() {}
+
+  util::optional<Result<V>> Pump() {
+    while (!finished_ && last_value_.has_value()) {
+      TransformFlow<V> next = transformer_(*last_value_);
+      if (next.ReadyForNext()) {
+        last_value_.reset();
+      }
+      if (next.Finished()) {
+        finished_ = true;
+      }
+      if (!next.Ok()) {
+        return next.status();
+      }
+      if (next.HasValue()) {
+        return next.Value();
+      }
+    }
+    if (finished_) {
+      return IterationTraits<V>::End();
+    }
+    return util::optional<V>();

Review comment:
       This doesn't match the return type of the function, please use `return {};` or `return util::nullopt;` to be more clear

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -556,6 +560,33 @@ inline bool WaitForAll(const std::vector<Future<T>*>& futures,
   return waiter->Wait(seconds);
 }
 
+template <typename T>

Review comment:
       ```suggestion
   /// \brief Create a Future which completes when all of `futures` complete.
   ///
   /// The future's result is a vector of the results of `futures`.
   /// Note that this future will never be marked "failed"; failed results
   /// will be stored in the result vector alongside successful results.
   template <typename T>
   ```

##########
File path: cpp/src/arrow/util/iterator.cc
##########
@@ -67,6 +67,8 @@ class ReadaheadQueue::Impl : public std::enable_shared_from_this<ReadaheadQueue:
   }
 
   Status PopDone(std::unique_ptr<ReadaheadPromise>* out) {
+    DCHECK_GT(max_readahead_, 0);  // This function has no purpose and should not be
+                                   // called if using the queue unbounded

Review comment:
       `DCHECK` macros (except `DCHECK_OK`) can be streamed into, which makes assert failures more helpful and searchable:
   ```suggestion
       DCHECK_GT(max_readahead_, 0) << "PopDone should never be called if using the queue unbounded";
   ```

##########
File path: cpp/src/arrow/util/future_test.cc
##########
@@ -832,6 +832,145 @@ TEST(FutureCompletionTest, FutureVoid) {
   }
 }
 
+TEST(FutureAllTest, Simple) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(2, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(2, *results[1]);
+  }));
+
+  // Finish in reverse order, results should still be delivered in proper order
+  AssertNotFinished(combined);
+  f2.MarkFinished(2);
+  AssertNotFinished(combined);
+  f1.MarkFinished(1);
+  AssertSuccessful(combined);
+}
+
+TEST(FutureAllTest, Failure) {
+  auto f1 = Future<int>::Make();
+  auto f2 = Future<int>::Make();
+  auto f3 = Future<int>::Make();
+  std::vector<Future<int>> futures = {f1, f2, f3};
+  auto combined = arrow::All(futures);
+
+  ARROW_UNUSED(combined.Then([](std::vector<Result<int>> results) {
+    ASSERT_EQ(3, results.size());
+    ASSERT_EQ(1, *results[0]);
+    ASSERT_EQ(Status::IOError("XYZ"), results[1].status());
+    ASSERT_EQ(3, *results[2]);
+  }));
+
+  f1.MarkFinished(1);
+  f2.MarkFinished(Status::IOError("XYZ"));
+  f3.MarkFinished(3);
+
+  AssertFinished(combined);
+}
+
+TEST(FutureLoopTest, Sync) {
+  struct {
+    int i = 0;
+    Future<int> Get() { return Future<int>::MakeFinished(i++); }
+  } IntSource;
+
+  bool do_fail = false;
+  std::vector<int> ints;
+  auto loop_body = [&] {
+    return IntSource.Get().Then([&](int i) -> Result<ControlFlow<int>> {
+      if (do_fail && i == 3) {
+        return Status::IOError("xxx");
+      }
+
+      if (i == 5) {
+        int sum = 0;
+        for (int i : ints) sum += i;
+        return Break(sum);
+      }
+
+      ints.push_back(i);
+      return Continue();
+    });
+  };
+
+  {
+    auto sum_fut = Loop(loop_body);
+    AssertSuccessful(sum_fut);
+
+    ASSERT_OK_AND_ASSIGN(auto sum, sum_fut.result());
+    ASSERT_EQ(sum, 0 + 1 + 2 + 3 + 4);
+  }
+
+  {
+    do_fail = true;
+    IntSource.i = 0;
+    auto sum_fut = Loop(loop_body);
+    AssertFailed(sum_fut);
+    ASSERT_RAISES(IOError, sum_fut.result());
+  }
+}
+
+TEST(FutureLoopTest, EmptyBreakValue) {
+  Future<> none_fut =
+      Loop([&] { return Future<>::MakeFinished().Then([&](...) { return Break(); }); });
+  AssertSuccessful(none_fut);
+}
+
+TEST(FutureLoopTest, MoveOnlyBreakValue) {
+  Future<MoveOnlyDataType> one_fut = Loop([&] {
+    return Future<int>::MakeFinished(1).Then(
+        [&](int i) { return Break(MoveOnlyDataType(i)); });
+  });
+  AssertSuccessful(one_fut);
+  ASSERT_OK_AND_ASSIGN(auto one, std::move(one_fut).result());
+  ASSERT_EQ(one, 1);
+}
+
+TEST(FutureLoopTest, StackOverflow) {
+  // Looping over futures is normally a rather recursive task.  If the futures complete
+  // synchronously (because they are already finished) it could lead to a stack overflow
+  // if care is not taken.
+  int counter = 0;
+  auto loop_body = [&counter]() -> Future<ControlFlow<int>> {
+    while (counter < 1000000) {
+      counter++;
+      return Future<ControlFlow<int>>::MakeFinished(Continue());

Review comment:
       Maybe it'd be useful to have an implicit constructor `Future<T>(Result<T>)` for finished futures, then I think we'd be able to just write
   ```c++
         return Continue();
   ```

##########
File path: cpp/src/arrow/util/future.h
##########
@@ -580,4 +611,73 @@ inline std::vector<int> WaitForAny(const std::vector<Future<T>*>& futures,
   return waiter->MoveFinishedFutures();
 }
 
+template <typename T = detail::Empty>
+struct ControlFlow {
+  using BreakValueType = T;
+
+  bool IsBreak() const { return break_value_.has_value(); }
+
+  static Result<BreakValueType> MoveBreakValue(const ControlFlow& cf) {
+    return std::move(*cf.break_value_);
+  }
+
+  mutable util::optional<BreakValueType> break_value_;
+};
+
+struct Continue {
+  template <typename T>
+  operator ControlFlow<T>() && {  // NOLINT explicit
+    return {};
+  }
+};
+
+template <typename T = detail::Empty>
+ControlFlow<T> Break(T break_value = {}) {
+  return ControlFlow<T>{std::move(break_value)};
+}
+
+template <typename Iterate,

Review comment:
       ```suggestion
   /// \brief Loop through an asynchronous sequence
   ///
   /// \param[in] iterate A generator of Future<ControlFlow<BreakValue>>. On completion of each yielded
   /// future the resulting ControlFlow will be examined. A Break will terminate the loop, while a Continue
   /// will re-invoke `iterate`.
   /// \return A future which will complete when a Future returned by iterate completes with a Break
   template <typename Iterate,
   ```

##########
File path: cpp/src/arrow/util/iterator_test.cc
##########
@@ -129,11 +130,47 @@ template <typename T>
 inline Iterator<T> EmptyIt() {
   return MakeEmptyIterator<T>();
 }
-
 inline Iterator<TestInt> VectorIt(std::vector<TestInt> v) {
   return MakeVectorIterator<TestInt>(std::move(v));
 }
 
+std::function<Future<TestInt>()> AsyncVectorIt(std::vector<TestInt> v) {
+  auto index = std::make_shared<size_t>(0);
+  auto vec = std::make_shared<std::vector<TestInt>>(std::move(v));
+  return [index, vec]() -> Future<TestInt> {
+    if (*index >= vec->size()) {
+      return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
+    }
+    auto next = (*vec)[*index];
+    (*index)++;
+    return Future<TestInt>::MakeFinished(next);
+  };

Review comment:
       ```suggestion
     size_t index = 0;
     return [index, v]() mutable -> Future<TestInt> {
       if (index >= vec.size()) {
         return Future<TestInt>::MakeFinished(IterationTraits<TestInt>::End());
       }
       return Future<TestInt>::MakeFinished(v[index++]);
     };
   ```




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