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


##########
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()) {
+        state_->cv_.wait(lock, [&] { return !state_->error_.ok(); });
+      }
+      return state_->error_;
+    }
+
+    ARROW_ASSIGN_OR_RAISE(
+        auto batch, ImportDeviceRecordBatch(&out, state_->schema_, 
state_->mapper_));
+    return RecordBatchWithMetadata{std::move(batch), std::move(task.second)};
+  }
+
+  static Future<std::shared_ptr<AsyncRecordBatchIterator::State>> Make(
+      AsyncRecordBatchIterator& iterator, struct 
ArrowAsyncDeviceStreamHandler* handler) {
+    auto iterator_fut = 
Future<std::shared_ptr<AsyncRecordBatchIterator::State>>::Make();
+
+    auto private_data = new PrivateData{iterator.state_};
+    private_data->fut_iterator_ = iterator_fut;
+
+    handler->private_data = private_data;
+    handler->on_schema = on_schema;
+    handler->on_next_task = on_next_task;
+    handler->on_error = on_error;
+    handler->release = release;
+    return iterator_fut;
+  }
+
+ private:
+  struct PrivateData {
+    explicit PrivateData(std::shared_ptr<State> state) : 
state_(std::move(state)) {}
+
+    std::shared_ptr<State> state_;
+    Future<std::shared_ptr<AsyncRecordBatchIterator::State>> fut_iterator_;
+    ARROW_DISALLOW_COPY_AND_ASSIGN(PrivateData);
+  };
+
+  static int on_schema(struct ArrowAsyncDeviceStreamHandler* self,
+                       struct ArrowSchema* stream_schema,
+                       const char* additional_metadata) {
+    auto* private_data = reinterpret_cast<PrivateData*>(self->private_data);
+    if (self->producer != nullptr) {
+      private_data->state_->producer_ = self->producer;
+      private_data->state_->device_type_ =
+          static_cast<DeviceAllocationType>(self->producer->device_type);
+    }
+
+    auto maybe_schema = ImportSchema(stream_schema);
+    if (!maybe_schema.ok()) {
+      private_data->fut_iterator_.MarkFinished(maybe_schema.status());
+      return EINVAL;
+    }
+
+    auto schema = maybe_schema.MoveValueUnsafe();
+    private_data->state_->schema_ = schema;
+    private_data->fut_iterator_.MarkFinished(private_data->state_);
+    self->producer->request(self->producer,
+                            
static_cast<int64_t>(private_data->state_->queue_size_));
+    return 0;
+  }
+
+  static int on_next_task(ArrowAsyncDeviceStreamHandler* self, ArrowAsyncTask* 
task,
+                          const char* metadata) {
+    auto* private_data = reinterpret_cast<PrivateData*>(self->private_data);
+
+    if (task == nullptr) {
+      std::unique_lock<std::mutex> lock(private_data->state_->mutex_);
+      private_data->state_->end_of_stream_ = true;
+      lock.unlock();
+      private_data->state_->cv_.notify_one();
+      return 0;
+    }
+
+    std::shared_ptr<KeyValueMetadata> kvmetadata;
+    if (metadata != nullptr) {
+      auto maybe_decoded = DecodeMetadata(metadata);
+      if (!maybe_decoded.ok()) {
+        private_data->state_->error_ = maybe_decoded.status();
+        private_data->state_->cv_.notify_one();
+        return EINVAL;
+      }
+
+      kvmetadata = maybe_decoded->metadata;
+    }
+
+    std::unique_lock<std::mutex> lock(private_data->state_->mutex_);
+    private_data->state_->batches_.emplace(*task, std::move(kvmetadata));
+    lock.unlock();
+    private_data->state_->cv_.notify_one();
+    return 0;
+  }
+
+  static void on_error(ArrowAsyncDeviceStreamHandler* self, int code, const 
char* message,
+                       const char* metadata) {
+    auto* private_data = reinterpret_cast<PrivateData*>(self->private_data);
+    std::unique_lock<std::mutex> lock(private_data->state_->mutex_);
+    private_data->state_->error_ = Status::FromDetailAndArgs(
+        StatusCode::UnknownError,
+        std::make_shared<AsyncErrorDetail>(code, message, metadata),
+        std::string(message));
+    lock.unlock();
+    private_data->state_->cv_.notify_one();
+  }
+
+  static void release(ArrowAsyncDeviceStreamHandler* self) {
+    auto* private_data = reinterpret_cast<PrivateData*>(self->private_data);
+    delete private_data;
+  }
+
+  std::shared_ptr<State> state_;
+};
+
+struct AsyncProducer {
+  struct State {
+    struct ArrowAsyncProducer producer_;
+
+    std::mutex mutex_;
+    std::condition_variable cv_;
+    uint64_t pending_requests_{0};
+    bool cancelled_{false};
+    Status error_{Status::OK()};
+  };
+
+  AsyncProducer(DeviceAllocationType device_type, struct ArrowSchema* schema,
+                struct ArrowAsyncDeviceStreamHandler* handler)
+      : handler_{handler}, state_{std::make_shared<State>()} {
+    state_->producer_.device_type = static_cast<ArrowDeviceType>(device_type);
+    state_->producer_.private_data = reinterpret_cast<void*>(state_.get());
+    state_->producer_.request = AsyncProducer::request;
+    state_->producer_.cancel = AsyncProducer::cancel;
+    handler_->producer = &state_->producer_;
+
+    if (int status = handler_->on_schema(handler_, schema, nullptr) != 0) {
+      state_->error_ =
+          Status::UnknownError("Received error from handler::on_schema ", 
status);
+    }
+  }
+
+  struct PrivateTaskData {
+    PrivateTaskData(std::shared_ptr<State> producer, 
std::shared_ptr<RecordBatch> record)
+        : producer_{std::move(producer)}, record_(std::move(record)) {}
+
+    std::shared_ptr<State> producer_;
+    std::shared_ptr<RecordBatch> record_;
+    ARROW_DISALLOW_COPY_AND_ASSIGN(PrivateTaskData);
+  };
+
+  Status operator()(const std::shared_ptr<RecordBatch>& record) {
+    std::unique_lock<std::mutex> lock(state_->mutex_);
+    if (state_->pending_requests_ == 0) {
+      state_->cv_.wait(lock, [this]() -> bool {

Review Comment:
   I suppose the way it "should" work is that we never call into the 
AsyncGenerator unless we have a pending request? But we'd then need to handle 
more of the control flow here ourselves



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