alamb commented on code in PR #16196:
URL: https://github.com/apache/datafusion/pull/16196#discussion_r2134852899


##########
datafusion/physical-optimizer/src/optimizer.rs:
##########
@@ -137,6 +138,7 @@ impl PhysicalOptimizer {
             // are not present, the load of executors such as join or union 
will be
             // reduced by narrowing their input tables.
             Arc::new(ProjectionPushdown::new()),
+            Arc::new(WrapLeaves::new()),

Review Comment:
   Can we please call this pass something related to Cancel or Yield? Like 
`InsertYieldExec` to make it clearer what it is doing?



##########
datafusion/core/tests/execution/infinite_cancel.rs:
##########
@@ -0,0 +1,856 @@
+// 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::error::Error;
+use std::fmt::Formatter;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll};
+
+use arrow::array::{Array, Int64Array, RecordBatch};
+use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef};
+use arrow_schema::SortOptions;
+use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, 
TaskContext};
+use datafusion::functions_aggregate::sum;
+use datafusion::physical_expr::aggregate::AggregateExprBuilder;
+use datafusion::physical_expr::{EquivalenceProperties, Partitioning};
+use datafusion::physical_plan::aggregates::{
+    AggregateExec, AggregateMode, PhysicalGroupBy,
+};
+use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
+use datafusion::physical_plan::{
+    DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties,
+};
+use datafusion::prelude::SessionContext;
+use datafusion::{common, physical_plan};
+use datafusion_common::config::ConfigOptions;
+use datafusion_common::{JoinType, ScalarValue};
+use datafusion_expr_common::operator::Operator::Gt;
+use datafusion_physical_expr::expressions::{col, BinaryExpr, Column, Literal};
+use datafusion_physical_expr_common::physical_expr::PhysicalExpr;
+use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr;
+use datafusion_physical_optimizer::wrap_leaves_cancellation::WrapLeaves;
+use datafusion_physical_optimizer::PhysicalOptimizerRule;
+use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec;
+use datafusion_physical_plan::filter::FilterExec;
+use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode, 
SortMergeJoinExec};
+use datafusion_physical_plan::projection::ProjectionExec;
+use datafusion_physical_plan::repartition::RepartitionExec;
+use datafusion_physical_plan::sorts::sort::SortExec;
+use datafusion_physical_plan::union::InterleaveExec;
+
+use futures::{Stream, StreamExt};
+use rstest::rstest;
+use tokio::select;
+
+struct InfiniteStream {
+    batch: RecordBatch,
+    poll_count: usize,
+}
+
+impl RecordBatchStream for InfiniteStream {
+    fn schema(&self) -> SchemaRef {
+        self.batch.schema()
+    }
+}
+
+impl Stream for InfiniteStream {
+    type Item = common::Result<RecordBatch>;
+
+    fn poll_next(
+        mut self: Pin<&mut Self>,
+        _cx: &mut Context<'_>,
+    ) -> Poll<Option<Self::Item>> {
+        self.poll_count += 1;
+        Poll::Ready(Some(Ok(self.batch.clone())))
+    }
+}
+
+#[derive(Debug)]
+struct InfiniteExec {

Review Comment:
   Instead of a new exec, perhaps we could use `MemoryExec` (with like 1000 
`batch.clone()` for example) to show it is configured correctly



##########
datafusion/sqllogictest/test_files/explain.slt:
##########
@@ -242,6 +242,7 @@ physical_plan after OutputRequirements DataSourceExec: 
file_groups={1 group: [[W
 physical_plan after LimitAggregation SAME TEXT AS ABOVE

Review Comment:
   I am confused about why YieldStreamExec does not appear in more of the 
explain `slt` plans



##########
datafusion/physical-plan/src/yield_stream.rs:
##########
@@ -0,0 +1,271 @@
+// 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::execution_plan::CardinalityEffect::{self, Equal};
+use crate::{
+    DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, 
RecordBatchStream,
+    SendableRecordBatchStream,
+};
+use arrow::record_batch::RecordBatch;
+use arrow_schema::Schema;
+use datafusion_common::{internal_err, Result, Statistics};
+use datafusion_execution::TaskContext;
+
+use futures::{Stream, StreamExt};
+
+/// An identity stream that passes batches through as is, but yields control
+/// back to the runtime every `period` batches. This stream is useful to
+/// construct a mechanism that allows operators that do not directly cooperate
+/// with the runtime to  check/support cancellation.
+pub struct YieldStream {
+    inner: SendableRecordBatchStream,
+    batches_processed: usize,
+    period: usize,
+}
+
+impl YieldStream {
+    pub fn new(inner: SendableRecordBatchStream, mut period: usize) -> Self {
+        if period == 0 {
+            period = usize::MAX;
+        }
+        Self {
+            inner,
+            batches_processed: 0,
+            period,
+        }
+    }
+}
+
+impl Stream for YieldStream {
+    type Item = Result<RecordBatch>;
+
+    fn poll_next(
+        mut self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<Option<Self::Item>> {
+        if self.batches_processed >= self.period {
+            self.batches_processed = 0;
+            cx.waker().wake_by_ref();
+            return Poll::Pending;
+        }
+
+        let value = self.inner.poll_next_unpin(cx);
+        match value {
+            Poll::Ready(Some(Ok(_))) => {
+                self.batches_processed += 1;
+            }
+            Poll::Pending => {
+                self.batches_processed = 0;
+            }
+            _ => {}
+        }
+        value
+    }
+}
+
+impl RecordBatchStream for YieldStream {
+    fn schema(&self) -> Arc<Schema> {
+        self.inner.schema()
+    }
+}
+
+/// This operator wraps any other execution plan and to "adapt" it to cooperate
+/// with the runtime by yielding control back to the runtime every `frequency`
+/// batches. This is useful for operators that do not natively support yielding
+/// control, allowing them to be used in a runtime that requires yielding for
+/// cancellation or other purposes.

Review Comment:
   I think it would also help to explain here how to avoid this Exec in your 
plan
   
   ```suggestion
   /// cancellation or other purposes.
   ///
   /// # Note
   /// If your ExecutionPlan periodically yields control back to the scheduler
   /// implement [`ExecutionPlan::with_cooperative_yields`] to avoid the need 
for this
   /// node.
   ```



##########
datafusion/physical-optimizer/src/wrap_leaves_cancellation.rs:
##########
@@ -0,0 +1,113 @@
+// 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 crate::PhysicalOptimizerRule;
+use datafusion_common::config::ConfigOptions;
+use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRecursion};
+use datafusion_common::Result;
+use datafusion_physical_plan::execution_plan::EmissionType;
+use datafusion_physical_plan::yield_stream::YieldStreamExec;
+use datafusion_physical_plan::ExecutionPlan;
+use std::fmt::{Debug, Formatter};
+use std::sync::Arc;
+
+/// WrapLeaves is a PhysicalOptimizerRule that finds every
+/// pipeline‐breaking node (emission_type == Final) and then
+/// wraps all of its leaf children in YieldStreamExec.
+pub struct WrapLeaves {}
+
+impl WrapLeaves {
+    pub fn new() -> Self {
+        Self {}
+    }
+
+    /// This function is called on every plan node during transform_down().
+    /// If the node is a leaf (no children), we wrap it in a new 
YieldStreamExec
+    /// and stop recursing further under that branch (TreeNodeRecursion::Jump).
+    fn wrap_leaves(
+        plan: Arc<dyn ExecutionPlan>,
+    ) -> Result<Transformed<Arc<dyn ExecutionPlan>>> {
+        if plan.children().is_empty() {
+            // Leaf: wrap it in YieldStreamExec, and do not descend further
+            let wrapped = Arc::new(YieldStreamExec::new(plan));
+            Ok(Transformed::new(
+                wrapped,
+                /* changed */ true,
+                TreeNodeRecursion::Jump,
+            ))
+        } else {
+            // Not a leaf: leave unchanged and keep recursing
+            Ok(Transformed::no(plan))
+        }
+    }
+
+    /// This function is called on every plan node during transform_down().
+    ///
+    /// If this node itself is a pipeline breaker (emission_type == Final),
+    /// we perform a second pass of transform_down with wrap_leaves. Then we
+    /// set TreeNodeRecursion::Jump so that we do not descend any deeper under
+    /// this subtree (we’ve already wrapped its leaves).
+    fn wrap_leaves_of_pipeline_breakers(
+        plan: Arc<dyn ExecutionPlan>,
+    ) -> Result<Transformed<Arc<dyn ExecutionPlan>>> {
+        let is_pipeline_breaker = plan.properties().emission_type == 
EmissionType::Final;

Review Comment:
   I missed this code in the initial 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.

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

Reply via email to