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


##########
cpp/src/arrow/c/bridge.cc:
##########
@@ -2511,4 +2516,333 @@ Result<std::shared_ptr<ChunkedArray>> 
ImportDeviceChunkedArray(
   return ImportChunked</*IsDevice=*/true>(stream, mapper);
 }
 
+namespace {
+
+class AsyncRecordBatchIterator {
+ public:
+  struct State {
+    State(uint64_t queue_size, const DeviceMemoryMapper mapper)
+        : queue_size_{queue_size}, mapper_{std::move(mapper)} {}
+
+    const uint64_t queue_size_;
+    const DeviceMemoryMapper mapper_;
+    ArrowAsyncProducer* producer_;
+    DeviceAllocationType device_type_;
+
+    std::mutex mutex_;
+    std::shared_ptr<Schema> schema_;
+    std::condition_variable cv_;
+    std::queue<std::pair<ArrowAsyncTask, std::shared_ptr<KeyValueMetadata>>> 
batches_;
+    bool end_of_stream_ = false;
+    Status error_{Status::OK()};
+  };
+
+  AsyncRecordBatchIterator(uint64_t queue_size, const DeviceMemoryMapper 
mapper)
+      : state_{std::make_shared<State>(queue_size, std::move(mapper))} {}
+
+  explicit AsyncRecordBatchIterator(std::shared_ptr<State> state)
+      : state_{std::move(state)} {}
+
+  const std::shared_ptr<Schema>& schema() const { return state_->schema_; }
+
+  DeviceAllocationType device_type() const { return state_->device_type_; }
+
+  Result<RecordBatchWithMetadata> Next() {

Review Comment:
   I guess blocking calls are OK due to MakeBackgroundGenerator here?



##########
cpp/src/arrow/c/bridge.cc:
##########
@@ -2511,4 +2516,333 @@ Result<std::shared_ptr<ChunkedArray>> 
ImportDeviceChunkedArray(
   return ImportChunked</*IsDevice=*/true>(stream, mapper);
 }
 
+namespace {
+
+class AsyncRecordBatchIterator {
+ public:
+  struct State {
+    State(uint64_t queue_size, const DeviceMemoryMapper mapper)
+        : queue_size_{queue_size}, mapper_{std::move(mapper)} {}
+
+    const uint64_t queue_size_;
+    const DeviceMemoryMapper mapper_;
+    ArrowAsyncProducer* producer_;
+    DeviceAllocationType device_type_;
+
+    std::mutex mutex_;
+    std::shared_ptr<Schema> schema_;
+    std::condition_variable cv_;
+    std::queue<std::pair<ArrowAsyncTask, std::shared_ptr<KeyValueMetadata>>> 
batches_;
+    bool end_of_stream_ = false;
+    Status error_{Status::OK()};
+  };
+
+  AsyncRecordBatchIterator(uint64_t queue_size, const DeviceMemoryMapper 
mapper)
+      : state_{std::make_shared<State>(queue_size, std::move(mapper))} {}
+
+  explicit AsyncRecordBatchIterator(std::shared_ptr<State> state)
+      : state_{std::move(state)} {}
+
+  const std::shared_ptr<Schema>& schema() const { return state_->schema_; }
+
+  DeviceAllocationType device_type() const { return state_->device_type_; }
+
+  Result<RecordBatchWithMetadata> Next() {
+    std::pair<ArrowAsyncTask, std::shared_ptr<KeyValueMetadata>> task;
+    {
+      std::unique_lock<std::mutex> lock(state_->mutex_);
+      state_->cv_.wait(lock, [&] {
+        return !state_->error_.ok() || !state_->batches_.empty() ||
+               state_->end_of_stream_;
+      });
+      if (!state_->error_.ok()) {
+        return state_->error_;
+      }
+
+      if (state_->batches_.empty() && state_->end_of_stream_) {
+        return RecordBatchWithMetadata{nullptr, nullptr};
+      }
+
+      task = state_->batches_.front();
+      state_->batches_.pop();
+    }
+
+    state_->producer_->request(state_->producer_, 1);
+    ArrowDeviceArray out;
+    if (task.first.extract_data(&task.first, &out) != 0) {
+      std::unique_lock<std::mutex> lock(state_->mutex_);
+      if (state_->error_.ok()) {

Review Comment:
   nit: I don't think you need this check? You can always wait on the CV with 
the condition



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

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

Reply via email to