pitrou commented on a change in pull request #9714:
URL: https://github.com/apache/arrow/pull/9714#discussion_r595143165
##########
File path: cpp/src/arrow/util/async_generator.h
##########
@@ -336,6 +345,68 @@ class ReadaheadGenerator {
std::queue<Future<T>> readahead_queue_;
};
+template <typename T>
+class PushGenerator {
+ struct State {
+ util::Mutex mutex;
+ std::deque<Result<T>> result_q;
+ util::optional<Future<T>> consumer_fut;
+ bool finished = false;
+ };
+
+ struct Generator {
+ const std::shared_ptr<State> state_;
+
+ Future<T> operator()() {
+ auto lock = state_->mutex.Lock();
+ assert(!state_->consumer_fut.has_value()); // Non-reentrant
+ if (!state_->result_q.empty()) {
+ auto fut =
Future<T>::MakeFinished(std::move(state_->result_q.front()));
+ state_->result_q.pop_front();
+ return fut;
+ }
+ if (state_->finished) {
+ return AsyncGeneratorEnd<T>();
+ }
+ auto fut = Future<T>::Make();
+ state_->consumer_fut = fut;
+ return fut;
+ }
+ };
+
+ public:
+ PushGenerator() : state_(std::make_shared<State>()) {}
+
+ void Push(Result<T> result) {
+ auto lock = state_->mutex.Lock();
+ if (state_->consumer_fut.has_value()) {
+ auto fut = std::move(state_->consumer_fut.value());
+ state_->consumer_fut.reset();
+ lock.Unlock(); // unlock before potentially invoking a callback
+ fut.MarkFinished(std::move(result));
+ return;
+ }
+ state_->result_q.push_back(std::move(result));
+ }
+
+ void Close() {
+ auto lock = state_->mutex.Lock();
+ state_->finished = true;
+ if (state_->consumer_fut.has_value()) {
+ auto fut = std::move(state_->consumer_fut.value());
+ state_->consumer_fut.reset();
+ lock.Unlock(); // unlock before potentially invoking a callback
+ fut.MarkFinished(IterationTraits<T>::End());
+ }
+ }
+
+ /// Return a non-reentrant async generator
+ Generator generator() { return Generator{state_}; }
Review comment:
Yes, I don't know how to make this API homogenous with the rest. I'll
add comments anyway.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]