zhuqi-lucas commented on code in PR #16196: URL: https://github.com/apache/datafusion/pull/16196#discussion_r2121426764
########## datafusion/physical-plan/src/yield_stream.rs: ########## @@ -0,0 +1,209 @@ +// 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. + +use std::any::Any; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use crate::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, +}; +use arrow::record_batch::RecordBatch; +use arrow_schema::Schema; +use datafusion_common::Result; +use datafusion_execution::TaskContext; +use futures::Stream; + +/// Number of batches to yield before voluntarily returning Pending. +/// This allows long-running operators to periodically yield control +/// back to the executor (e.g., to handle cancellation). +const YIELD_BATCHES: usize = 64; + +/// A stream that yields batches of data, yielding control back to the executor every `YIELD_BATCHES` batches +/// +/// This can be useful to allow operators that might not yield to check for cancellation +pub struct YieldStream { + inner: SendableRecordBatchStream, + batches_processed: usize, + buffer: Option<Result<RecordBatch>>, +} + +impl YieldStream { + pub fn new(inner: SendableRecordBatchStream) -> Self { + Self { + inner, + batches_processed: 0, + buffer: None, + } + } +} + +// Stream<Item = Result<RecordBatch>> to poll_next_unpin +impl Stream for YieldStream { + type Item = Result<RecordBatch>; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll<Option<Self::Item>> { + let this = &mut *self; + + if let Some(batch) = this.buffer.take() { + return Poll::Ready(Some(batch)); + } + + // Instead of `poll_next_unpin`, use `Pin::new(&mut this.inner).poll_next(cx)` + match Pin::new(&mut this.inner).poll_next(cx) { + Poll::Ready(Some(Ok(batch))) => { + this.batches_processed += 1; + if this.batches_processed >= YIELD_BATCHES { + this.batches_processed = 0; + // We need to buffer the batch when we return Poll::Pending, + // so that we can return it on the next poll. + // Otherwise, the next poll will miss the batch and return None. + this.buffer = Some(Ok(batch)); + cx.waker().wake_by_ref(); + Poll::Pending + } else { + Poll::Ready(Some(Ok(batch))) + } + } + other => other, Review Comment: Good suggestion, thank you @pepijnve ! -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org