diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index bea970f144863..b500b6d9d9aa1 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -1303,10 +1303,8 @@ mod test { } fn make_dynamic_expr(expr: Arc) -> Arc { - Arc::new(DynamicFilterPhysicalExpr::new( - expr.children().into_iter().map(Arc::clone).collect(), - expr, - )) + let children = expr.children().into_iter().map(Arc::clone).collect(); + Arc::new(DynamicFilterPhysicalExpr::new(expr, children)) } #[tokio::test] diff --git a/datafusion/execution/src/metrics/mod.rs b/datafusion/execution/src/metrics/mod.rs index e76ba1de38a9a..638e66b490aa8 100644 --- a/datafusion/execution/src/metrics/mod.rs +++ b/datafusion/execution/src/metrics/mod.rs @@ -89,9 +89,9 @@ pub struct Metric { /// will be shown. /// - When set to `summary`, only metrics with type `MetricType::Summary` are shown. /// -/// # Difference from `EXPLAIN ANALYZE VERBOSE`: -/// The `VERBOSE` keyword controls whether per-partition metrics are shown (when specified), -/// or aggregated metrics are displayed (when omitted). +/// # Difference from `EXPLAIN ANALYZE VERBOSE`: +/// The `VERBOSE` keyword controls whether per-partition metrics are shown (when specified), +/// or aggregated metrics are displayed (when omitted). /// In contrast, the `analyze_level` configuration determines which categories or /// levels of metrics are displayed. #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] @@ -213,6 +213,16 @@ impl MetricsSet { Default::default() } + /// Return a number of metrics. + pub fn len(&self) -> usize { + self.metrics.len() + } + + /// Check if the set is empty. + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + /// Add the specified metric pub fn push(&mut self, metric: Arc) { self.metrics.push(metric) diff --git a/datafusion/physical-expr/src/expressions/dynamic_filters.rs b/datafusion/physical-expr/src/expressions/dynamic_filters.rs index 615d9cbbf61ac..a44cdb35c876c 100644 --- a/datafusion/physical-expr/src/expressions/dynamic_filters.rs +++ b/datafusion/physical-expr/src/expressions/dynamic_filters.rs @@ -22,11 +22,11 @@ use tokio::sync::watch; use crate::PhysicalExpr; use arrow::datatypes::{DataType, Schema}; use datafusion_common::{ - Result, + Result, internal_err, tree_node::{Transformed, TransformedResult, TreeNode}, }; use datafusion_expr::ColumnarValue; -use datafusion_physical_expr_common::physical_expr::{DynEq, DynHash}; +use datafusion_physical_expr_common::physical_expr::DynHash; /// State of a dynamic filter, tracking both updates and completion. #[derive(Debug, Clone, Copy, PartialEq, Eq)] @@ -46,90 +46,48 @@ impl FilterState { } } -/// A dynamic [`PhysicalExpr`] that can be updated by anyone with a reference to it. -/// -/// Any `ExecutionPlan` that uses this expression and holds a reference to it internally should probably also -/// implement `ExecutionPlan::reset_state` to remain compatible with recursive queries and other situations where -/// the same `ExecutionPlan` is reused with different data. -#[derive(Debug)] -pub struct DynamicFilterPhysicalExpr { +/// Planning time version of dynamic filter, not intended to be concurrently modified. +/// Typically, it is converted to the [`DynamicFilterPhysicalExpr`] when a plan is executed. +#[derive(Debug, Clone)] +pub struct PlannedDynamicFilterPhysicalExpr { + pub(super) expr: Arc, /// The original children of this PhysicalExpr, if any. /// This is necessary because the dynamic filter may be initialized with a placeholder (e.g. `lit(true)`) /// and later remapped to the actual expressions that are being filtered. /// But we need to know the children (e.g. columns referenced in the expression) ahead of time to evaluate the expression correctly. - children: Vec>, + pub(super) children: Vec>, /// If any of the children were remapped / modified (e.g. to adjust for projections) we need to keep track of the new children - /// so that when we update `current()` in subsequent iterations we can re-apply the replacements. - remapped_children: Option>>, - /// The source of dynamic filters. - inner: Arc>, - /// Broadcasts filter state (updates and completion) to all waiters. - state_watch: watch::Sender, - /// For testing purposes track the data type and nullability to make sure they don't change. - /// If they do, there's a bug in the implementation. - /// But this can have overhead in production, so it's only included in our tests. - data_type: Arc>>, - nullable: Arc>>, -} - -#[derive(Debug)] -struct Inner { - /// A counter that gets incremented every time the expression is updated so that we can track changes cheaply. - /// This is used for [`PhysicalExpr::snapshot_generation`] to have a cheap check for changes. - generation: u64, - expr: Arc, - /// Flag for quick synchronous check if filter is complete. - /// This is redundant with the watch channel state, but allows us to return immediately - /// from `wait_complete()` without subscribing if already complete. - is_complete: bool, + /// so that when we update [`DynamicFilterPhysicalExpr::current`] in subsequent iterations we can re-apply the replacements. + pub(super) remapped_children: Option>>, } -impl Inner { - fn new(expr: Arc) -> Self { - Self { - // Start with generation 1 which gives us a different result for [`PhysicalExpr::generation`] than the default 0. - // This is not currently used anywhere but it seems useful to have this simple distinction. - generation: 1, - expr, - is_complete: false, - } - } - - /// Clone the inner expression. - fn expr(&self) -> &Arc { - &self.expr +impl PartialEq for PlannedDynamicFilterPhysicalExpr { + fn eq(&self, other: &Self) -> bool { + self.children == other.children && self.expr.dyn_eq(&other.expr) } } -impl Hash for DynamicFilterPhysicalExpr { +impl Hash for PlannedDynamicFilterPhysicalExpr { fn hash(&self, state: &mut H) { - let inner = self.current().expect("Failed to get current expression"); - inner.dyn_hash(state); + self.expr.dyn_hash(state); self.children.dyn_hash(state); self.remapped_children.dyn_hash(state); } } -impl PartialEq for DynamicFilterPhysicalExpr { - fn eq(&self, other: &Self) -> bool { - let inner = self.current().expect("Failed to get current expression"); - let our_children = self.remapped_children.as_ref().unwrap_or(&self.children); - let other_children = other.remapped_children.as_ref().unwrap_or(&other.children); - let other = other.current().expect("Failed to get current expression"); - inner.dyn_eq(other.as_any()) && our_children == other_children - } -} - -impl Eq for DynamicFilterPhysicalExpr {} +impl Eq for PlannedDynamicFilterPhysicalExpr {} -impl Display for DynamicFilterPhysicalExpr { +impl Display for PlannedDynamicFilterPhysicalExpr { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - self.render(f, |expr, f| write!(f, "{expr}")) + write!(f, "DynamicFilter [ ")?; + // Same as init-generation [`DynamicFilterPhysicalExpr`]. + format_empty_filter(f)?; + write!(f, " ]") } } -impl DynamicFilterPhysicalExpr { - /// Create a new [`DynamicFilterPhysicalExpr`] +impl PlannedDynamicFilterPhysicalExpr { + /// Create a new [`PlannedDynamicFilterPhysicalExpr`] /// from an initial expression and a list of children. /// The list of children is provided separately because /// the initial expression may not have the same children. @@ -157,20 +115,169 @@ impl DynamicFilterPhysicalExpr { /// /// [`collect_columns`]: crate::utils::collect_columns pub fn new( + expr: Arc, children: Vec>, - inner: Arc, ) -> Self { - let (state_watch, _) = watch::channel(FilterState::InProgress { generation: 1 }); Self { children, + expr, remapped_children: None, // Initially no remapped children - inner: Arc::new(RwLock::new(Inner::new(inner))), + } + } + + /// Convert this origin into [`DynamicFilterPhysicalExpr`] which could be + /// modified concurrently and tracks self generation. + pub fn to_executable(&self) -> DynamicFilterPhysicalExpr { + DynamicFilterPhysicalExpr::init( + self.children.clone(), + Arc::clone(&self.expr), + self.remapped_children.clone(), + ) + } +} + +impl PhysicalExpr for PlannedDynamicFilterPhysicalExpr { + fn as_any(&self) -> &dyn Any { + self + } + + fn evaluate(&self, _batch: &arrow::array::RecordBatch) -> Result { + internal_err!("planned expr is not supposed to be directly evaluated") + } + + fn children(&self) -> Vec<&Arc> { + self.remapped_children + .as_ref() + .unwrap_or(&self.children) + .iter() + .collect() + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + Ok(Arc::new(Self { + expr: Arc::clone(&self.expr), + children: self.children.clone(), + remapped_children: Some(children), + })) + } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.fmt(f) + } +} + +/// A dynamic [`PhysicalExpr`] that can be updated by anyone with a reference to it. +#[derive(Debug, Clone)] +pub struct DynamicFilterPhysicalExpr { + /// The same as [`PlannedDynamicFilterPhysicalExpr::children`]. + children: Vec>, + /// The source of dynamic filters. + inner: Arc>, + /// Broadcasts filter state (updates and completion) to all waiters. + state_watch: watch::Sender, + /// For testing purposes track the data type and nullability to make sure they don't change. + /// If they do, there's a bug in the implementation. + /// But this can have overhead in production, so it's only included in our tests. + data_type: Arc>>, + nullable: Arc>>, + /// Origin expression. + origin: Arc, + /// Originated from [`PlannedDynamicFilterPhysicalExpr::remapped_children`]. + remapped_children: Option>>, +} + +impl Hash for DynamicFilterPhysicalExpr { + fn hash(&self, state: &mut H) { + let inner = self.current().expect("Failed to get current expression"); + inner.dyn_hash(state); + self.children.hash(state); + self.remapped_children.dyn_hash(state); + } +} + +impl PartialEq for DynamicFilterPhysicalExpr { + fn eq(&self, other: &Self) -> bool { + let inner = self.current().expect("Failed to get current expression"); + let our_children = self.remapped_children.as_ref().unwrap_or(&self.children); + let other_children = other.remapped_children.as_ref().unwrap_or(&other.children); + let other = other.current().expect("Failed to get current expression"); + inner.dyn_eq(other.as_any()) && our_children == other_children + } +} + +impl Eq for DynamicFilterPhysicalExpr {} + +impl Display for DynamicFilterPhysicalExpr { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.render(f, |expr, f| write!(f, "{expr}")) + } +} + +impl From<&PlannedDynamicFilterPhysicalExpr> for DynamicFilterPhysicalExpr { + fn from(planned: &PlannedDynamicFilterPhysicalExpr) -> Self { + planned.to_executable() + } +} + +#[derive(Debug)] +struct Inner { + /// A counter that gets incremented every time the expression is updated so that we can track changes cheaply. + /// This is used for [`PhysicalExpr::snapshot_generation`] to have a cheap check for changes. + generation: u64, + expr: Arc, + /// Flag for quick synchronous check if filter is complete. + /// This is redundant with the watch channel state, but allows us to return immediately + /// from `wait_complete()` without subscribing if already complete. + is_complete: bool, +} + +impl Inner { + fn new(expr: Arc) -> Self { + Self { + // Start with generation 1 which gives us a different result for [`PhysicalExpr::generation`] than the default 0. + // This is not currently used anywhere but it seems useful to have this simple distinction. + generation: 1, + expr, + is_complete: false, + } + } + + /// Clone the inner expression. + fn expr(&self) -> &Arc { + &self.expr + } +} + +impl DynamicFilterPhysicalExpr { + fn init( + children: Vec>, + origin: Arc, + remapped_children: Option>>, + ) -> Self { + let (state_watch, _) = watch::channel(FilterState::InProgress { generation: 1 }); + Self { + children, + remapped_children, + inner: Arc::new(RwLock::new(Inner::new(Arc::clone(&origin)))), state_watch, data_type: Arc::new(RwLock::new(None)), nullable: Arc::new(RwLock::new(None)), + origin, } } + /// Make a new [`DynamicFilterPhysicalExpr`]. + /// See the comment for [`PlannedDynamicFilterPhysicalExpr`] for details. + pub fn new( + origin: Arc, + children: Vec>, + ) -> Self { + Self::init(children, origin, None) + } + fn remap_children( children: &[Arc], remapped_children: Option<&Vec>>, @@ -294,6 +401,23 @@ impl DynamicFilterPhysicalExpr { .await; } + /// Check if this dynamic filter is originated from passed planned + /// [`PlannedDynamicFilterPhysicalExpr`] and if it is so -- return + /// its executable version. + pub fn as_dynamic_for( + &self, + origin: &PlannedDynamicFilterPhysicalExpr, + ) -> Option> { + if Arc::ptr_eq(&self.origin, &origin.expr) { + let mut expr = self.clone(); + expr.remapped_children = origin.remapped_children.clone(); + expr.children = origin.children.clone(); + Some(Arc::new(expr)) + } else { + None + } + } + fn render( &self, f: &mut std::fmt::Formatter<'_>, @@ -306,7 +430,7 @@ impl DynamicFilterPhysicalExpr { let current_generation = self.current_generation(); write!(f, "DynamicFilter [ ")?; if current_generation == 1 { - write!(f, "empty")?; + format_empty_filter(f)?; } else { render_expr(inner, f)?; } @@ -315,6 +439,10 @@ impl DynamicFilterPhysicalExpr { } } +fn format_empty_filter(f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "empty") +} + impl PhysicalExpr for DynamicFilterPhysicalExpr { fn as_any(&self) -> &dyn Any { self @@ -339,6 +467,7 @@ impl PhysicalExpr for DynamicFilterPhysicalExpr { state_watch: self.state_watch.clone(), data_type: Arc::clone(&self.data_type), nullable: Arc::clone(&self.nullable), + origin: Arc::clone(&self.origin), })) } @@ -442,10 +571,13 @@ mod test { datafusion_expr::Operator::Eq, lit(42) as Arc, )); - let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new( - vec![col("a", &table_schema).unwrap()], - expr as Arc, - )); + let dynamic_filter = Arc::new( + PlannedDynamicFilterPhysicalExpr::new( + expr as Arc, + vec![col("a", &table_schema).unwrap()], + ) + .to_executable(), + ); // Simulate two `ParquetSource` files with different filter schemas // Both of these should hit the same inner `PhysicalExpr` even after `update()` is called // and be able to remap children independently. @@ -539,7 +671,9 @@ mod test { #[test] fn test_snapshot() { let expr = lit(42) as Arc; - let dynamic_filter = DynamicFilterPhysicalExpr::new(vec![], Arc::clone(&expr)); + let dynamic_filter = + PlannedDynamicFilterPhysicalExpr::new(Arc::clone(&expr), vec![]) + .to_executable(); // Take a snapshot of the current expression let snapshot = dynamic_filter.snapshot().unwrap(); @@ -555,8 +689,11 @@ mod test { #[test] fn test_dynamic_filter_physical_expr_misbehaves_data_type_nullable() { - let dynamic_filter = - DynamicFilterPhysicalExpr::new(vec![], lit(42) as Arc); + let dynamic_filter = PlannedDynamicFilterPhysicalExpr::new( + lit(42) as Arc, + vec![], + ) + .to_executable(); // First call to data_type and nullable should set the initial values. let initial_data_type = dynamic_filter.data_type(&Schema::empty()).unwrap(); @@ -596,10 +733,13 @@ mod test { #[tokio::test] async fn test_wait_complete_already_complete() { - let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new( - vec![], - lit(42) as Arc, - )); + let dynamic_filter = Arc::new( + PlannedDynamicFilterPhysicalExpr::new( + lit(42) as Arc, + vec![], + ) + .to_executable(), + ); // Mark as complete immediately dynamic_filter.mark_complete(); @@ -631,10 +771,13 @@ mod test { )); // Create DynamicFilterPhysicalExpr with children [col_a, col_b] - let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new( - vec![Arc::clone(&col_a), Arc::clone(&col_b)], - expr as Arc, - )); + let dynamic_filter = Arc::new( + PlannedDynamicFilterPhysicalExpr::new( + expr as Arc, + vec![Arc::clone(&col_a), Arc::clone(&col_b)], + ) + .to_executable(), + ); // Clone the Arc (two references to the same DynamicFilterPhysicalExpr) let clone_1 = Arc::clone(&dynamic_filter); diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index c9e02708d6c28..a0f02184997d2 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -45,7 +45,7 @@ pub use cast::{CastExpr, cast}; pub use cast_column::CastColumnExpr; pub use column::{Column, col, with_new_schema}; pub use datafusion_expr::utils::format_state_name; -pub use dynamic_filters::DynamicFilterPhysicalExpr; +pub use dynamic_filters::{DynamicFilterPhysicalExpr, PlannedDynamicFilterPhysicalExpr}; pub use in_list::{InListExpr, in_list}; pub use is_not_null::{IsNotNullExpr, is_not_null}; pub use is_null::{IsNullExpr, is_null}; diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 68e67fa018f08..a73420928330a 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -42,6 +42,9 @@ force_hash_collisions = [] test_utils = ["arrow/test_utils"] tokio_coop = [] tokio_coop_fallback = [] +# Force physical plans to keep state separately from itself, making +# them re-executable. +stateless_plan = [] [lib] name = "datafusion_physical_plan" @@ -89,6 +92,10 @@ tokio = { workspace = true, features = [ harness = false name = "partial_ordering" +[[bench]] +harness = false +name = "plan_reuse" + [[bench]] harness = false name = "spill_io" diff --git a/datafusion/physical-plan/benches/plan_reuse.rs b/datafusion/physical-plan/benches/plan_reuse.rs new file mode 100644 index 0000000000000..c7a6182fe7ab3 --- /dev/null +++ b/datafusion/physical-plan/benches/plan_reuse.rs @@ -0,0 +1,192 @@ +use std::sync::{Arc, LazyLock}; + +use arrow_schema::{DataType, Field, Fields, Schema, SchemaRef}; +use criterion::{Criterion, criterion_group, criterion_main}; +use datafusion_common::Result; +use datafusion_common::tree_node::{Transformed, TreeNode}; +use datafusion_execution::TaskContext; +use datafusion_expr::Operator; +use datafusion_functions_aggregate::average::avg_udaf; +use datafusion_physical_expr::aggregate::AggregateExprBuilder; +use datafusion_physical_expr::expressions::{self, binary, lit}; +use datafusion_physical_expr::{Partitioning, PhysicalExpr}; +use datafusion_physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::{ + ExecutionPlan, execute_stream, filter::FilterExec, test::TestMemoryExec, +}; + +const NUM_FIELDS: usize = 1000; + +static SCHEMA: LazyLock = LazyLock::new(|| { + Arc::new(Schema::new( + (0..NUM_FIELDS) + .map(|i| Arc::new(Field::new(format!("x_{i}"), DataType::Int64, false))) + .collect::(), + )) +}); + +fn partitioning() -> Partitioning { + Partitioning::RoundRobinBatch(16) +} + +fn col_name(i: usize) -> String { + format!("x_{i}") +} + +fn aggr_name(i: usize) -> String { + format!("aggr({})", col_name(i)) +} + +fn col(i: usize) -> Arc { + expressions::col(&col_name(i), &SCHEMA).unwrap() +} + +/// Returns a typical plan for the query like: +/// +/// ```sql +/// SELECT aggr1(col1) as aggr1, aggr2(col2) as aggr2 FROM t +/// WHERE p1 +/// HAVING p2 +/// ``` +/// +/// A plan looks like: +/// +/// ```text +/// ProjectionExec +/// FilterExec +/// AggregateExec: mode=Final +/// CoalescePartitionsExec +/// AggregateExec: mode=Partial +/// RepartitionExec +/// FilterExec +/// TestMemoryExec +/// ``` +/// +fn query1_plan() -> Result> { + let schema = Arc::clone(&SCHEMA); + let input = TestMemoryExec::try_new(&[vec![]], Arc::clone(&schema), None)?; + + let plan = FilterExec::try_new( + // Some predicate. + binary( + binary(col(0), Operator::Eq, col(1), &schema)?, + Operator::And, + binary(col(2), Operator::Eq, lit(42_i64), &schema)?, + &schema, + )?, + Arc::new(input), + )?; + + let plan = RepartitionExec::try_new(Arc::new(plan), partitioning())?; + + let plan = { + // Partial aggregation. + let aggr_expr = (0..NUM_FIELDS) + .map(|i| { + AggregateExprBuilder::new(avg_udaf(), vec![col(i)]) + .schema(Arc::clone(&schema)) + .alias(aggr_name(i)) + .build() + .map(Arc::new) + }) + .collect::>>()?; + let filter_expr = (0..aggr_expr.len()).map(|_| None).collect(); + + AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::new(vec![], vec![], vec![], false), + aggr_expr, + filter_expr, + Arc::new(plan), + Arc::clone(&schema), + )? + }; + + let plan = CoalescePartitionsExec::new(Arc::new(plan)); + + let schema = plan.schema(); + let plan = { + // Final aggregation. + let aggr_expr = (0..NUM_FIELDS) + .map(|i| { + AggregateExprBuilder::new( + avg_udaf(), + vec![Arc::new(expressions::Column::new(&aggr_name(i), i))], + ) + .schema(Arc::clone(&schema)) + .alias(aggr_name(i)) + .build() + .map(Arc::new) + }) + .collect::>>()?; + let filter_expr = (0..aggr_expr.len()).map(|_| None).collect(); + + AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::new(vec![], vec![], vec![], false), + aggr_expr, + filter_expr, + Arc::new(plan), + Arc::clone(&schema), + )? + }; + + let schema = plan.schema(); + let plan = { + let predicate = (0..schema.fields.len()).fold(lit(true), |expr, i| { + binary( + expr, + Operator::And, + binary( + Arc::new(expressions::Column::new(schema.field(i).name(), i)), + Operator::Gt, + lit(i as i64), + &schema, + ) + .unwrap(), + &schema, + ) + .unwrap() + }); + + FilterExec::try_new(predicate, Arc::new(plan))? + }; + + Ok(Arc::new(plan)) +} + +#[cfg(not(feature = "stateless_plan"))] +fn reset_plan_states(plan: Arc) -> Arc { + plan.transform_up(|plan| { + let new_plan = Arc::clone(&plan).reset_state()?; + Ok(Transformed::yes(new_plan)) + }) + .unwrap() + .data +} + +fn bench_plan_execute(c: &mut Criterion) { + let task_ctx = Arc::new(TaskContext::default()); + let plan = query1_plan().unwrap(); + let rt = tokio::runtime::Runtime::new().unwrap(); + + c.bench_function("execute", |b| { + b.iter(|| { + #[cfg(not(feature = "stateless_plan"))] + let plan = reset_plan_states(Arc::clone(&plan)); + + #[cfg(feature = "stateless_plan")] + let plan = Arc::clone(&plan); + + let _stream = + rt.block_on(async { execute_stream(plan, Arc::clone(&task_ctx)) }); + }); + }); +} + +criterion_group!(benches, bench_plan_execute); +criterion_main!(benches); diff --git a/datafusion/physical-plan/src/aggregates/group_values/metrics.rs b/datafusion/physical-plan/src/aggregates/group_values/metrics.rs index b6c32204e85f0..ad02e59c0419a 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/metrics.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/metrics.rs @@ -50,10 +50,11 @@ impl GroupByMetrics { #[cfg(test)] mod tests { + use crate::ExecutionPlan; use crate::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; + use crate::execution_plan::collect_and_get_metrics_of; use crate::metrics::MetricsSet; use crate::test::TestMemoryExec; - use crate::{ExecutionPlan, collect}; use arrow::array::{Float64Array, UInt32Array}; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; @@ -126,7 +127,7 @@ mod tests { ), ]; - let aggregate_exec = Arc::new(AggregateExec::try_new( + let aggregate_exec: Arc = Arc::new(AggregateExec::try_new( AggregateMode::Partial, group_by, aggregates, @@ -136,10 +137,14 @@ mod tests { )?); let task_ctx = Arc::new(TaskContext::default()); - let _result = - collect(Arc::clone(&aggregate_exec) as _, Arc::clone(&task_ctx)).await?; - - let metrics = aggregate_exec.metrics().unwrap(); + let (_result, metrics) = collect_and_get_metrics_of( + Arc::clone(&aggregate_exec), + &aggregate_exec, + Arc::clone(&task_ctx), + ) + .await?; + + let metrics = metrics.unwrap(); assert_groupby_metrics(&metrics); Ok(()) @@ -193,7 +198,7 @@ mod tests { )?); // Create final aggregate - let final_aggregate = Arc::new(AggregateExec::try_new( + let final_aggregate: Arc = Arc::new(AggregateExec::try_new( AggregateMode::Final, group_by.as_final(), aggregates, @@ -203,10 +208,14 @@ mod tests { )?); let task_ctx = Arc::new(TaskContext::default()); - let _result = - collect(Arc::clone(&final_aggregate) as _, Arc::clone(&task_ctx)).await?; - - let metrics = final_aggregate.metrics().unwrap(); + let (_result, metrics) = collect_and_get_metrics_of( + Arc::clone(&final_aggregate), + &final_aggregate, + Arc::clone(&task_ctx), + ) + .await?; + + let metrics = metrics.unwrap(); assert_groupby_metrics(&metrics); Ok(()) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 06f12a90195d2..db7ae9c6ddebf 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -25,12 +25,16 @@ use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, }; +use crate::dynamic_filter::{ + DynamicFilterPhysicalExpr, PlannedDynamicFilterPhysicalExpr, +}; use crate::execution_plan::{CardinalityEffect, EmissionType}; use crate::filter_pushdown::{ ChildFilterDescription, ChildPushdownResult, FilterDescription, FilterPushdownPhase, FilterPushdownPropagation, PushedDownPredicate, }; -use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayFormatType, Distribution, ExecutionPlan, InputOrderMode, SendableRecordBatchStream, Statistics, @@ -49,10 +53,12 @@ use datafusion_common::{ Constraint, Constraints, Result, ScalarValue, assert_eq_or_internal_err, not_impl_err, }; use datafusion_execution::TaskContext; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_expr::{Accumulator, Aggregate}; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr::equivalence::ProjectionMapping; -use datafusion_physical_expr::expressions::{Column, DynamicFilterPhysicalExpr, lit}; +use datafusion_physical_expr::expressions::{Column, lit}; use datafusion_physical_expr::{ ConstExpr, EquivalenceProperties, physical_exprs_contains, }; @@ -465,6 +471,30 @@ struct AggrDynFilter { supported_accumulators_info: Vec, } +/// Planning time version of [`AggrDynFilter`]. +#[derive(Debug)] +struct PlannedAggrDynFilter { + filter: Arc, + supported_accumulators_info: Vec, +} + +impl PlannedAggrDynFilter { + fn to_executable(&self) -> AggrDynFilter { + AggrDynFilter { + #[cfg(feature = "stateless_plan")] + filter: Arc::new(self.filter.to_executable()), + // Filter is already executable. + #[cfg(not(feature = "stateless_plan"))] + filter: Arc::clone(&self.filter), + supported_accumulators_info: self + .supported_accumulators_info + .iter() + .map(PlannedPerAccumulatorDynFilter::to_executable) + .collect(), + } + } +} + // ---- Aggregate Dynamic Filter Utility Structs ---- /// Aggregate expressions that support the dynamic filter pushdown in aggregation. @@ -482,8 +512,27 @@ struct PerAccumulatorDynFilter { shared_bound: Arc>, } +/// Planning time version of the [`PerAccumulatorDynFilter`]. +#[derive(Debug)] +struct PlannedPerAccumulatorDynFilter { + aggr_type: DynamicFilterAggregateType, + /// During planning and optimization, the parent structure is kept in `AggregateExec`, + /// this index is into `aggr_expr` vec inside `AggregateExec`. + aggr_index: usize, +} + +impl PlannedPerAccumulatorDynFilter { + fn to_executable(&self) -> PerAccumulatorDynFilter { + PerAccumulatorDynFilter { + aggr_type: self.aggr_type, + aggr_index: self.aggr_index, + shared_bound: Arc::new(Mutex::new(ScalarValue::Null)), + } + } +} + /// Aggregate types that are supported for dynamic filter in `AggregateExec` -#[derive(Debug, Clone)] +#[derive(Debug, Clone, Copy)] enum DynamicFilterAggregateType { Min, Max, @@ -512,8 +561,6 @@ pub struct AggregateExec { /// We need the input schema of partial aggregate to be able to deserialize aggregate /// expressions from protobuf for final aggregate. pub input_schema: SchemaRef, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, required_input_ordering: Option, /// Describes how the input is ordered relative to the group by columns input_order_mode: InputOrderMode, @@ -524,9 +571,44 @@ pub struct AggregateExec { /// During filter pushdown optimization, if a child node can accept this filter, /// it remains `Some(..)` to enable dynamic filtering during aggregate execution; /// otherwise, it is cleared to `None`. - dynamic_filter: Option>, + dynamic_filter: Option>, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, +} + +#[cfg(feature = "stateless_plan")] +mod exec_state { + use super::{AggrDynFilter, Any, Arc, DynamicFilterPhysicalExpr}; + use crate::state::PlanState; + + #[derive(Debug, Default, Clone)] + pub struct AggregateExecState { + /// Originated from [`AggregateExec::dynamic_filter`] on execution stage. + /// It is not stored in the case of stateful plan as in this case planned and executable + /// filters are the same and filter is stored directly in [`AggregateExec`]. + #[cfg(feature = "stateless_plan")] + pub dynamic_filter: Option>, + } + + impl PlanState for AggregateExecState { + fn as_any(&self) -> &dyn Any { + self + } + + fn dynamic_filters(&self) -> Vec> { + if let Some(dynamic_filter) = self.dynamic_filter.as_ref() { + vec![Arc::clone(&dynamic_filter.filter)] + } else { + vec![] + } + } + } } +#[cfg(feature = "stateless_plan")] +use exec_state::AggregateExecState; + impl AggregateExec { /// Function used in `OptimizeAggregateOrder` optimizer rule, /// where we need parts of the new value, others cloned from the old one @@ -539,7 +621,6 @@ impl AggregateExec { aggr_expr, // clone the rest of the fields required_input_ordering: self.required_input_ordering.clone(), - metrics: ExecutionPlanMetricsSet::new(), input_order_mode: self.input_order_mode.clone(), cache: self.cache.clone(), mode: self.mode, @@ -550,6 +631,8 @@ impl AggregateExec { schema: Arc::clone(&self.schema), input_schema: Arc::clone(&self.input_schema), dynamic_filter: self.dynamic_filter.clone(), + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), } } @@ -674,12 +757,13 @@ impl AggregateExec { input, schema, input_schema, - metrics: ExecutionPlanMetricsSet::new(), required_input_ordering, limit: None, input_order_mode, cache, dynamic_filter: None, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), }; exec.init_dynamic_filter(); @@ -736,10 +820,35 @@ impl AggregateExec { &self, partition: usize, context: &Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { if self.group_by.is_true_no_grouping() { + // Make an executable filter and put it into state. + #[cfg(feature = "stateless_plan")] + let dynamic_filter = state + .get_or_init_state(|| AggregateExecState { + dynamic_filter: self + .dynamic_filter + .as_ref() + .map(|filter| Arc::new(filter.to_executable())), + }) + .dynamic_filter + .clone(); + + // Get filter from self. + #[cfg(not(feature = "stateless_plan"))] + let dynamic_filter = self + .dynamic_filter + .clone() + .map(|f| Arc::new(f.to_executable())); + return Ok(StreamType::AggregateStream(AggregateStream::new( - self, context, partition, + self, + context, + partition, + dynamic_filter, + #[cfg(feature = "stateless_plan")] + state, )?)); } @@ -748,13 +857,24 @@ impl AggregateExec { && !self.is_unordered_unfiltered_group_by_distinct() { return Ok(StreamType::GroupedPriorityQueue( - GroupedTopKAggregateStream::new(self, context, partition, limit)?, + GroupedTopKAggregateStream::new( + self, + context, + partition, + limit, + #[cfg(feature = "stateless_plan")] + state, + )?, )); } // grouping by something else and we need to just materialize all results Ok(StreamType::GroupedHash(GroupedHashAggregateStream::new( - self, context, partition, + self, + context, + partition, + #[cfg(feature = "stateless_plan")] + state, )?)) } @@ -988,17 +1108,19 @@ impl AggregateExec { && arg.as_any().is::() { all_cols.push(Arc::clone(arg)); - aggr_dyn_filters.push(PerAccumulatorDynFilter { + aggr_dyn_filters.push(PlannedPerAccumulatorDynFilter { aggr_type, aggr_index: i, - shared_bound: Arc::new(Mutex::new(ScalarValue::Null)), }); } } if !aggr_dyn_filters.is_empty() { - self.dynamic_filter = Some(Arc::new(AggrDynFilter { - filter: Arc::new(DynamicFilterPhysicalExpr::new(all_cols, lit(true))), + self.dynamic_filter = Some(Arc::new(PlannedAggrDynFilter { + filter: Arc::new(PlannedDynamicFilterPhysicalExpr::new( + lit(true), + all_cols, + )), supported_accumulators_info: aggr_dyn_filters, })) } @@ -1224,11 +1346,18 @@ impl ExecutionPlan for AggregateExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { - self.execute_typed(partition, &context) - .map(|stream| stream.into()) + self.execute_typed( + partition, + &context, + #[cfg(feature = "stateless_plan")] + state, + ) + .map(|stream| stream.into()) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -1808,6 +1937,8 @@ mod tests { use crate::common; use crate::common::collect; use crate::execution_plan::Boundedness; + use crate::execution_plan::collect_and_get_metrics_of; + use crate::execution_plan::execute_plan; use crate::expressions::col; use crate::metrics::MetricValue; use crate::test::TestMemoryExec; @@ -1824,6 +1955,7 @@ mod tests { use datafusion_common::{DataFusionError, ScalarValue, internal_err}; use datafusion_execution::config::SessionConfig; use datafusion_execution::memory_pool::FairSpillPool; + use datafusion_execution::metrics::MetricsSet; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_functions_aggregate::array_agg::array_agg_udaf; use datafusion_functions_aggregate::average::avg_udaf; @@ -1986,7 +2118,7 @@ mod tests { Arc::new(TaskContext::default()) }; - let partial_aggregate = Arc::new(AggregateExec::try_new( + let partial_aggregate: Arc = Arc::new(AggregateExec::try_new( AggregateMode::Partial, grouping_set.clone(), aggregates.clone(), @@ -1995,8 +2127,12 @@ mod tests { Arc::clone(&input_schema), )?); - let result = - collect(partial_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; + let result = collect(execute_plan( + Arc::clone(&partial_aggregate), + 0, + Arc::clone(&task_ctx), + )?) + .await?; if spill { // In spill mode, we test with the limited memory, if the mem usage exceeds, @@ -2066,7 +2202,7 @@ mod tests { task_ctx }; - let merged_aggregate = Arc::new(AggregateExec::try_new( + let merged_aggregate: Arc = Arc::new(AggregateExec::try_new( AggregateMode::Final, final_grouping_set, aggregates, @@ -2075,7 +2211,12 @@ mod tests { input_schema, )?); - let result = collect(merged_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; + let (result, metrics) = collect_and_get_metrics_of( + Arc::clone(&merged_aggregate), + &merged_aggregate, + Arc::clone(&task_ctx), + ) + .await?; let batch = concat_batches(&result[0].schema(), &result)?; assert_eq!(batch.num_columns(), 4); assert_eq!(batch.num_rows(), 12); @@ -2104,7 +2245,7 @@ mod tests { ); } - let metrics = merged_aggregate.metrics().unwrap(); + let metrics = metrics.unwrap(); let output_rows = metrics.output_rows().unwrap(); assert_eq!(12, output_rows); @@ -2136,7 +2277,7 @@ mod tests { Arc::new(TaskContext::default()) }; - let partial_aggregate = Arc::new(AggregateExec::try_new( + let partial_aggregate: Arc = Arc::new(AggregateExec::try_new( AggregateMode::Partial, grouping_set.clone(), aggregates.clone(), @@ -2145,8 +2286,12 @@ mod tests { Arc::clone(&input_schema), )?); - let result = - collect(partial_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; + let result = collect(execute_plan( + Arc::clone(&partial_aggregate), + 0, + Arc::clone(&task_ctx), + )?) + .await?; if spill { allow_duplicates! { @@ -2180,7 +2325,7 @@ mod tests { let final_grouping_set = grouping_set.as_final(); - let merged_aggregate = Arc::new(AggregateExec::try_new( + let merged_aggregate: Arc = Arc::new(AggregateExec::try_new( AggregateMode::Final, final_grouping_set, aggregates, @@ -2199,7 +2344,12 @@ mod tests { } else { Arc::clone(&task_ctx) }; - let result = collect(merged_aggregate.execute(0, task_ctx)?).await?; + let (result, metrics) = collect_and_get_metrics_of( + Arc::clone(&merged_aggregate), + &merged_aggregate, + Arc::clone(&task_ctx), + ) + .await?; let batch = concat_batches(&result[0].schema(), &result)?; assert_eq!(batch.num_columns(), 2); assert_eq!(batch.num_rows(), 3); @@ -2218,7 +2368,7 @@ mod tests { // For row 3: 4, (3 + 4 + 4) / 3 } - let metrics = merged_aggregate.metrics().unwrap(); + let metrics = metrics.unwrap(); let output_rows = metrics.output_rows().unwrap(); let spill_count = metrics.spill_count().unwrap(); let spilled_bytes = metrics.spilled_bytes().unwrap(); @@ -2316,6 +2466,7 @@ mod tests { &self, _partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { let stream = if self.yield_first { TestYieldingStream::New @@ -2494,6 +2645,14 @@ mod tests { Arc::clone(&input_schema), )?); + #[cfg(feature = "stateless_plan")] + let state = + PlanStateNode::new_root_arc(Arc::clone(&partial_aggregate) as Arc<_>); + + #[cfg(feature = "stateless_plan")] + let stream = partial_aggregate.execute_typed(0, &task_ctx, &state)?; + + #[cfg(not(feature = "stateless_plan"))] let stream = partial_aggregate.execute_typed(0, &task_ctx)?; // ensure that we really got the version we wanted @@ -2943,8 +3102,12 @@ mod tests { schema, )?); - let output = - collect(aggregate_exec.execute(0, Arc::new(TaskContext::default()))?).await?; + let output = collect(execute_plan( + aggregate_exec, + 0, + Arc::new(TaskContext::default()), + )?) + .await?; allow_duplicates! { assert_snapshot!(batches_to_sort_string(&output), @r" @@ -3059,7 +3222,7 @@ mod tests { let session_config = SessionConfig::default(); let ctx = TaskContext::default().with_session_config(session_config); - let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; + let output = collect(execute_plan(aggregate_exec, 0, Arc::new(ctx))?).await?; allow_duplicates! { assert_snapshot!(batches_to_string(&output), @r" @@ -3134,7 +3297,7 @@ mod tests { ); let ctx = TaskContext::default().with_session_config(session_config); - let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; + let output = collect(execute_plan(aggregate_exec, 0, Arc::new(ctx))?).await?; allow_duplicates! { assert_snapshot!(batches_to_string(&output), @r" @@ -3221,7 +3384,7 @@ mod tests { ); let ctx = TaskContext::default().with_session_config(session_config); - let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; + let output = collect(execute_plan(aggregate_exec, 0, Arc::new(ctx))?).await?; allow_duplicates! { assert_snapshot!(batches_to_string(&output), @r" @@ -3344,7 +3507,7 @@ mod tests { ), ]; - let single_aggregate = Arc::new(AggregateExec::try_new( + let single_aggregate: Arc = Arc::new(AggregateExec::try_new( AggregateMode::Single, grouping_set, aggregates, @@ -3365,9 +3528,14 @@ mod tests { )), ); - let result = collect(single_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; + let (result, metrics) = collect_and_get_metrics_of( + Arc::clone(&single_aggregate), + &single_aggregate, + Arc::clone(&task_ctx), + ) + .await?; - assert_spill_count_metric(expect_spill, single_aggregate); + assert_spill_count_metric(expect_spill, metrics); allow_duplicates! { assert_snapshot!(batches_to_string(&result), @r" @@ -3384,11 +3552,8 @@ mod tests { Ok(()) } - fn assert_spill_count_metric( - expect_spill: bool, - single_aggregate: Arc, - ) { - if let Some(metrics_set) = single_aggregate.metrics() { + fn assert_spill_count_metric(expect_spill: bool, metrics: Option) { + if let Some(metrics_set) = metrics { let mut spill_count = 0; // Inspect metrics for SpillCount @@ -3489,7 +3654,7 @@ mod tests { ), ]; - let single_aggregate = Arc::new(AggregateExec::try_new( + let single_aggregate: Arc = Arc::new(AggregateExec::try_new( AggregateMode::Single, grouping_set, aggregates, @@ -3510,10 +3675,15 @@ mod tests { )), ); - let result = collect(single_aggregate.execute(0, Arc::clone(&task_ctx))?).await; + let result = collect_and_get_metrics_of( + Arc::clone(&single_aggregate), + &single_aggregate, + Arc::clone(&task_ctx), + ) + .await; match result { - Ok(result) => { - assert_spill_count_metric(true, single_aggregate); + Ok((result, metrics)) => { + assert_spill_count_metric(true, metrics); allow_duplicates! { assert_snapshot!(batches_to_string(&result), @r" @@ -3649,7 +3819,7 @@ mod tests { .unwrap(), ])?; - let aggr = Arc::new(AggregateExec::try_new( + let aggr: Arc = Arc::new(AggregateExec::try_new( AggregateMode::Single, PhysicalGroupBy::new( vec![ @@ -3672,8 +3842,10 @@ mod tests { )?); let task_ctx = new_spill_ctx(1, 600); - let result = collect(aggr.execute(0, Arc::clone(&task_ctx))?).await?; - assert_spill_count_metric(true, aggr); + let (result, metrics) = + collect_and_get_metrics_of(Arc::clone(&aggr), &aggr, Arc::clone(&task_ctx)) + .await?; + assert_spill_count_metric(true, metrics); allow_duplicates! { assert_snapshot!(batches_to_string(&result), @r" diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index a55d70ca6fb27..1cbe2492df8c9 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -22,6 +22,8 @@ use crate::aggregates::{ aggregate_expressions, create_accumulators, finalize_aggregation, }; use crate::metrics::{BaselineMetrics, RecordOutput}; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; @@ -274,12 +276,19 @@ impl AggregateStream { agg: &AggregateExec, context: &Arc, partition: usize, + maybe_dynamic_filter: Option>, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + let agg_schema = Arc::clone(&agg.schema); let agg_filter_expr = agg.filter_expr.clone(); - let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); - let input = agg.input.execute(partition, Arc::clone(context))?; + let baseline_metrics = BaselineMetrics::new(plan_metrics!(agg, state), partition); + let input = execute_input!(0, agg.input, partition, Arc::clone(context), state)?; let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; let filter_expressions = match agg.mode { @@ -299,19 +308,16 @@ impl AggregateStream { // 1. AggregateExec did the check and ensure it supports the dynamic filter // (its dynamic_filter field will be Some(..)) // 2. Aggregate dynamic filter is enabled from the config - let mut maybe_dynamic_filter = match agg.dynamic_filter.as_ref() { - Some(filter) => Some(Arc::clone(filter)), - _ => None, - }; - - if !context + let maybe_dynamic_filter = if !context .session_config() .options() .optimizer .enable_aggregate_dynamic_filter_pushdown { - maybe_dynamic_filter = None; - } + None + } else { + maybe_dynamic_filter + }; let inner = AggregateStreamInner { schema: Arc::clone(&agg.schema), diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index cb22fbf9a06a1..3bb210cb8539f 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -33,6 +33,8 @@ use crate::metrics::{BaselineMetrics, MetricBuilder, RecordOutput}; use crate::sorts::sort::sort_batch; use crate::sorts::streaming_merge::{SortedSpillFile, StreamingMergeBuilder}; use crate::spill::spill_manager::SpillManager; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{PhysicalExpr, aggregates, metrics}; use crate::{RecordBatchStream, SendableRecordBatchStream}; @@ -462,16 +464,22 @@ impl GroupedHashAggregateStream { agg: &AggregateExec, context: &Arc, partition: usize, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + debug!("Creating GroupedHashAggregateStream"); let agg_schema = Arc::clone(&agg.schema); let agg_group_by = agg.group_by.clone(); let agg_filter_expr = agg.filter_expr.clone(); let batch_size = context.session_config().batch_size(); - let input = agg.input.execute(partition, Arc::clone(context))?; - let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); - let group_by_metrics = GroupByMetrics::new(&agg.metrics, partition); + let input = execute_input!(0, agg.input, partition, Arc::clone(context), state)?; + let baseline_metrics = BaselineMetrics::new(plan_metrics!(agg, state), partition); + let group_by_metrics = GroupByMetrics::new(plan_metrics!(agg, state), partition); let timer = baseline_metrics.elapsed_compute().timer(); @@ -603,7 +611,7 @@ impl GroupedHashAggregateStream { let spill_manager = SpillManager::new( context.runtime_env(), - metrics::SpillMetrics::new(&agg.metrics, partition), + metrics::SpillMetrics::new(plan_metrics!(agg, state), partition), Arc::clone(&spill_schema), ) .with_compression_type(context.session_config().spill_compression()); @@ -615,7 +623,7 @@ impl GroupedHashAggregateStream { is_stream_merging: false, merging_aggregate_arguments, merging_group_by: PhysicalGroupBy::new_single(merging_group_by_expr), - peak_mem_used: MetricBuilder::new(&agg.metrics) + peak_mem_used: MetricBuilder::new(plan_metrics!(agg, state)) .gauge("peak_mem_used", partition), spill_manager, }; @@ -639,7 +647,7 @@ impl GroupedHashAggregateStream { options.skip_partial_aggregation_probe_rows_threshold; let probe_ratio_threshold = options.skip_partial_aggregation_probe_ratio_threshold; - let skipped_aggregation_rows = MetricBuilder::new(&agg.metrics) + let skipped_aggregation_rows = MetricBuilder::new(plan_metrics!(agg, state)) .counter("skipped_aggregation_rows", partition); Some(SkipAggregationProbe::new( probe_rows_threshold, @@ -652,7 +660,7 @@ impl GroupedHashAggregateStream { let reduction_factor = if agg.mode == AggregateMode::Partial { Some( - MetricBuilder::new(&agg.metrics) + MetricBuilder::new(plan_metrics!(agg, state)) .with_type(metrics::MetricType::SUMMARY) .ratio_metrics("reduction_factor", partition), ) @@ -1305,7 +1313,8 @@ impl GroupedHashAggregateStream { #[cfg(test)] mod tests { use super::*; - use crate::execution_plan::ExecutionPlan; + use crate::ExecutionPlan; + use crate::execution_plan::{execute_plan, execute_plan_and_get_metrics_of}; use crate::test::TestMemoryExec; use arrow::array::{Int32Array, Int64Array}; use arrow::datatypes::{DataType, Field, Schema}; @@ -1389,18 +1398,17 @@ mod tests { let exec = Arc::new(TestMemoryExec::update_cache(&Arc::new(exec))); // Use Partial mode where the race condition occurs - let aggregate_exec = AggregateExec::try_new( + let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, PhysicalGroupBy::new_single(group_expr), aggr_expr, vec![None], exec, Arc::clone(&schema), - )?; + )?); // Execute and collect results - let mut stream = - GroupedHashAggregateStream::new(&aggregate_exec, &Arc::clone(&task_ctx), 0)?; + let mut stream = execute_plan(aggregate_exec, 0, task_ctx)?; let mut results = Vec::new(); while let Some(result) = stream.next().await { @@ -1532,18 +1540,21 @@ mod tests { let exec = Arc::new(TestMemoryExec::update_cache(&Arc::new(exec))); // Use Partial mode - let aggregate_exec = AggregateExec::try_new( + let aggregate_exec: Arc = Arc::new(AggregateExec::try_new( AggregateMode::Partial, PhysicalGroupBy::new_single(group_expr), aggr_expr, vec![None], exec, Arc::clone(&schema), - )?; + )?); - // Execute and collect results - let mut stream = - GroupedHashAggregateStream::new(&aggregate_exec, &Arc::clone(&task_ctx), 0)?; + let (mut stream, metrics) = execute_plan_and_get_metrics_of( + Arc::clone(&aggregate_exec), + &aggregate_exec, + 0, + task_ctx, + )?; let mut results = Vec::new(); while let Some(result) = stream.next().await { @@ -1553,7 +1564,7 @@ mod tests { // Check that skip aggregation actually happened // The key metric is skipped_aggregation_rows - let metrics = aggregate_exec.metrics().unwrap(); + let metrics = metrics.unwrap(); let skipped_rows = metrics .sum_by_name("skipped_aggregation_rows") .map(|m| m.as_usize()) diff --git a/datafusion/physical-plan/src/aggregates/topk_stream.rs b/datafusion/physical-plan/src/aggregates/topk_stream.rs index 1096eb64d3ae7..b6ac02639b7d8 100644 --- a/datafusion/physical-plan/src/aggregates/topk_stream.rs +++ b/datafusion/physical-plan/src/aggregates/topk_stream.rs @@ -24,6 +24,8 @@ use crate::aggregates::{ evaluate_many, }; use crate::metrics::BaselineMetrics; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::{Array, ArrayRef, RecordBatch}; use arrow::datatypes::SchemaRef; @@ -57,12 +59,19 @@ impl GroupedTopKAggregateStream { context: &Arc, partition: usize, limit: usize, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + let agg_schema = Arc::clone(&aggr.schema); let group_by = aggr.group_by.clone(); - let input = aggr.input.execute(partition, Arc::clone(context))?; - let baseline_metrics = BaselineMetrics::new(&aggr.metrics, partition); - let group_by_metrics = GroupByMetrics::new(&aggr.metrics, partition); + let input = execute_input!(0, aggr.input, partition, Arc::clone(context), state)?; + let baseline_metrics = + BaselineMetrics::new(plan_metrics!(aggr, state), partition); + let group_by_metrics = GroupByMetrics::new(plan_metrics!(aggr, state), partition); let aggregate_arguments = aggregate_expressions(&aggr.aggr_expr, &aggr.mode, group_by.expr.len())?; let (val_field, desc) = aggr diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 1fb8f93a38782..509cd43e8bd4c 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -27,6 +27,8 @@ use super::{ }; use crate::display::DisplayableExecutionPlan; use crate::metrics::MetricType; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; @@ -160,6 +162,7 @@ impl ExecutionPlan for AnalyzeExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { assert_eq_or_internal_err!( partition, @@ -174,7 +177,18 @@ impl ExecutionPlan for AnalyzeExec { let mut builder = RecordBatchReceiverStream::builder(self.schema(), num_input_partitions); + #[cfg(feature = "stateless_plan")] + let child_state = state.child_state(0); for input_partition in 0..num_input_partitions { + #[cfg(feature = "stateless_plan")] + builder.run_input( + Arc::clone(&self.input), + input_partition, + Arc::clone(&context), + &child_state, + ); + + #[cfg(not(feature = "stateless_plan"))] builder.run_input( Arc::clone(&self.input), input_partition, @@ -194,6 +208,9 @@ impl ExecutionPlan for AnalyzeExec { // JoinSet that computes the overall row count and final // record batch let mut input_stream = builder.build(); + + #[cfg(feature = "stateless_plan")] + let state = Arc::clone(state); let output = async move { let mut total_rows = 0; while let Some(batch) = input_stream.next().await.transpose()? { @@ -202,13 +219,14 @@ impl ExecutionPlan for AnalyzeExec { let duration = Instant::now() - start; create_output_batch( - verbose, - show_statistics, + (verbose, show_statistics), total_rows, duration, &captured_input, &captured_schema, &metric_types, + #[cfg(feature = "stateless_plan")] + state, ) }; @@ -221,13 +239,13 @@ impl ExecutionPlan for AnalyzeExec { /// Creates the output of AnalyzeExec as a RecordBatch fn create_output_batch( - verbose: bool, - show_statistics: bool, + (verbose, show_statistics): (bool, bool), total_rows: usize, duration: std::time::Duration, input: &Arc, schema: &SchemaRef, metric_types: &[MetricType], + #[cfg(feature = "stateless_plan")] state: Arc, ) -> Result { let mut type_builder = StringBuilder::with_capacity(1, 1024); let mut plan_builder = StringBuilder::with_capacity(1, 1024); @@ -235,7 +253,14 @@ fn create_output_batch( // TODO use some sort of enum rather than strings? type_builder.append_value("Plan with Metrics"); - let annotated_plan = DisplayableExecutionPlan::with_metrics(input.as_ref()) + #[cfg(feature = "stateless_plan")] + let annotated_plan = + DisplayableExecutionPlan::with_metrics(input.as_ref(), Arc::clone(&state)); + + #[cfg(not(feature = "stateless_plan"))] + let annotated_plan = DisplayableExecutionPlan::with_metrics(input.as_ref()); + + let annotated_plan = annotated_plan .set_metric_types(metric_types.to_vec()) .set_show_statistics(show_statistics) .indent(verbose) @@ -247,7 +272,14 @@ fn create_output_batch( if verbose { type_builder.append_value("Plan with Full Metrics"); - let annotated_plan = DisplayableExecutionPlan::with_full_metrics(input.as_ref()) + #[cfg(feature = "stateless_plan")] + let annotated_plan = + DisplayableExecutionPlan::with_full_metrics(input.as_ref(), state); + + #[cfg(not(feature = "stateless_plan"))] + let annotated_plan = DisplayableExecutionPlan::with_full_metrics(input.as_ref()); + + let annotated_plan = annotated_plan .set_metric_types(metric_types.to_vec()) .set_show_statistics(show_statistics) .indent(verbose) diff --git a/datafusion/physical-plan/src/async_func.rs b/datafusion/physical-plan/src/async_func.rs index 280995f480042..ec2d6311ed085 100644 --- a/datafusion/physical-plan/src/async_func.rs +++ b/datafusion/physical-plan/src/async_func.rs @@ -15,7 +15,8 @@ // specific language governing permissions and limitations // under the License. -use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::stream::RecordBatchStreamAdapter; use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PlanProperties, @@ -24,6 +25,8 @@ use arrow::array::RecordBatch; use arrow_schema::{Fields, Schema, SchemaRef}; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRecursion}; use datafusion_common::{Result, assert_eq_or_internal_err}; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::ScalarFunctionExpr; use datafusion_physical_expr::async_scalar_function::AsyncFuncExpr; @@ -46,6 +49,7 @@ pub struct AsyncFuncExec { async_exprs: Vec>, input: Arc, cache: PlanProperties, + #[cfg(not(feature = "stateless_plan"))] metrics: ExecutionPlanMetricsSet, } @@ -80,6 +84,7 @@ impl AsyncFuncExec { input, async_exprs, cache, + #[cfg(not(feature = "stateless_plan"))] metrics: ExecutionPlanMetricsSet::new(), }) } @@ -171,7 +176,13 @@ impl ExecutionPlan for AsyncFuncExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::execute_input; + trace!( "Start AsyncFuncExpr::execute for partition {} of context session_id {} and task_id {:?}", partition, @@ -181,7 +192,8 @@ impl ExecutionPlan for AsyncFuncExec { // TODO figure out how to record metrics // first execute the input stream - let input_stream = self.input.execute(partition, Arc::clone(&context))?; + let input_stream = + execute_input!(0, self.input, partition, Arc::clone(&context), state)?; // now, for each record batch, evaluate the async expressions and add the columns to the result let async_exprs_captured = Arc::new(self.async_exprs.clone()); @@ -216,6 +228,7 @@ impl ExecutionPlan for AsyncFuncExec { Ok(Box::pin(adapter)) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 13bb862ab9371..e4882a72f94eb 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -22,9 +22,11 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use super::metrics::BaselineMetrics; use super::{DisplayAs, ExecutionPlanProperties, PlanProperties, Statistics}; use crate::projection::ProjectionExec; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, }; @@ -33,6 +35,8 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::TaskContext; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_expr::PhysicalExpr; use crate::coalesce::{LimitedBatchCoalescer, PushBatchStatus}; @@ -65,9 +69,10 @@ pub struct CoalesceBatchesExec { target_batch_size: usize, /// Maximum number of rows to fetch, `None` means fetching all rows fetch: Option, + cache: PlanProperties, /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] metrics: ExecutionPlanMetricsSet, - cache: PlanProperties, } impl CoalesceBatchesExec { @@ -78,8 +83,9 @@ impl CoalesceBatchesExec { input, target_batch_size, fetch: None, - metrics: ExecutionPlanMetricsSet::new(), cache, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), } } @@ -182,19 +188,26 @@ impl ExecutionPlan for CoalesceBatchesExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + Ok(Box::pin(CoalesceBatchesStream { - input: self.input.execute(partition, context)?, + input: execute_input!(0, self.input, partition, context, state)?, coalescer: LimitedBatchCoalescer::new( self.input.schema(), self.target_batch_size, self.fetch, ), - baseline_metrics: BaselineMetrics::new(&self.metrics, partition), + baseline_metrics: BaselineMetrics::new(plan_metrics!(self, state), partition), completed: false, })) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -214,6 +227,7 @@ impl ExecutionPlan for CoalesceBatchesExec { input: Arc::clone(&self.input), target_batch_size: self.target_batch_size, fetch: limit, + #[cfg(not(feature = "stateless_plan"))] metrics: self.metrics.clone(), cache: self.cache.clone(), })) diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index d83f90eb3d8c1..3365d94a5f44c 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -21,7 +21,7 @@ use std::any::Any; use std::sync::Arc; -use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use super::metrics::BaselineMetrics; use super::stream::{ObservedStream, RecordBatchReceiverStream}; use super::{ DisplayAs, ExecutionPlanProperties, PlanProperties, SendableRecordBatchStream, @@ -31,12 +31,16 @@ use crate::execution_plan::{CardinalityEffect, EvaluationType, SchedulingType}; use crate::filter_pushdown::{FilterDescription, FilterPushdownPhase}; use crate::projection::{ProjectionExec, make_with_child}; use crate::sort_pushdown::SortOrderPushdownResult; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use datafusion_common::config::ConfigOptions; use datafusion_common::{Result, assert_eq_or_internal_err, internal_err}; use datafusion_execution::TaskContext; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_expr::PhysicalExpr; /// Merge execution plan executes partitions in parallel and combines them into a single @@ -45,11 +49,12 @@ use datafusion_physical_expr::PhysicalExpr; pub struct CoalescePartitionsExec { /// Input execution plan input: Arc, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, cache: PlanProperties, /// Optional number of rows to fetch. Stops producing rows after this fetch pub(crate) fetch: Option, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } impl CoalescePartitionsExec { @@ -58,9 +63,10 @@ impl CoalescePartitionsExec { let cache = Self::compute_properties(&input); CoalescePartitionsExec { input, - metrics: ExecutionPlanMetricsSet::new(), cache, fetch: None, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), } } @@ -160,7 +166,13 @@ impl ExecutionPlan for CoalescePartitionsExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + // CoalescePartitionsExec produces a single partition assert_eq_or_internal_err!( partition, @@ -176,9 +188,10 @@ impl ExecutionPlan for CoalescePartitionsExec { 1 => { // single-partition path: execute child directly, but ensure fetch is respected // (wrap with ObservedStream only if fetch is present so we don't add overhead otherwise) - let child_stream = self.input.execute(0, context)?; + let child_stream = execute_input!(0, self.input, 0, context, state)?; if self.fetch.is_some() { - let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + let baseline_metrics = + BaselineMetrics::new(plan_metrics!(self, state), partition); return Ok(Box::pin(ObservedStream::new( child_stream, baseline_metrics, @@ -188,7 +201,8 @@ impl ExecutionPlan for CoalescePartitionsExec { Ok(child_stream) } _ => { - let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + let baseline_metrics = + BaselineMetrics::new(plan_metrics!(self, state), partition); // record the (very) minimal work done so that // elapsed_compute is not reported as 0 let elapsed_compute = baseline_metrics.elapsed_compute().clone(); @@ -202,11 +216,23 @@ impl ExecutionPlan for CoalescePartitionsExec { // spawn independent tasks whose resulting streams (of batches) // are sent to the channel for consumption. + + #[cfg(feature = "stateless_plan")] + let child_state = state.child_state(0); for part_i in 0..input_partitions { + #[cfg(not(feature = "stateless_plan"))] + builder.run_input( + Arc::clone(&self.input), + part_i, + Arc::clone(&context), + ); + + #[cfg(feature = "stateless_plan")] builder.run_input( Arc::clone(&self.input), part_i, Arc::clone(&context), + &child_state, ); } @@ -220,6 +246,7 @@ impl ExecutionPlan for CoalescePartitionsExec { } } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -273,6 +300,7 @@ impl ExecutionPlan for CoalescePartitionsExec { Some(Arc::new(CoalescePartitionsExec { input: Arc::clone(&self.input), fetch: limit, + #[cfg(not(feature = "stateless_plan"))] metrics: self.metrics.clone(), cache: self.cache.clone(), })) @@ -327,6 +355,7 @@ impl ExecutionPlan for CoalescePartitionsExec { #[cfg(test)] mod tests { use super::*; + use crate::execution_plan::execute_plan; use crate::test::exec::{ BlockingExec, PanicExec, assert_strong_count_converges_to_zero, }; @@ -356,7 +385,7 @@ mod tests { ); // the result should contain 4 batches (one per input partition) - let iter = merge.execute(0, task_ctx)?; + let iter = execute_plan(Arc::new(merge), 0, task_ctx)?; let batches = common::collect(iter).await?; assert_eq!(batches.len(), num_partitions); @@ -412,7 +441,7 @@ mod tests { // Test with fetch=3 let coalesce = CoalescePartitionsExec::new(input).with_fetch(Some(3)); - let stream = coalesce.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(coalesce), 0, task_ctx)?; let batches = common::collect(stream).await?; let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum(); @@ -432,7 +461,7 @@ mod tests { // Test with fetch=1 (the original bug: was returning multiple rows instead of 1) let coalesce = CoalescePartitionsExec::new(input).with_fetch(Some(1)); - let stream = coalesce.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(coalesce), 0, task_ctx)?; let batches = common::collect(stream).await?; let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum(); @@ -454,7 +483,7 @@ mod tests { // Test without fetch (should return all rows) let coalesce = CoalescePartitionsExec::new(input); - let stream = coalesce.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(coalesce), 0, task_ctx)?; let batches = common::collect(stream).await?; let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum(); @@ -476,7 +505,7 @@ mod tests { // Test with fetch larger than available rows let coalesce = CoalescePartitionsExec::new(input).with_fetch(Some(200)); - let stream = coalesce.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(coalesce), 0, task_ctx)?; let batches = common::collect(stream).await?; let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum(); @@ -499,7 +528,7 @@ mod tests { // Test with fetch=400 (exactly all rows) let coalesce = CoalescePartitionsExec::new(csv).with_fetch(Some(400)); - let stream = coalesce.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(coalesce), 0, task_ctx)?; let batches = common::collect(stream).await?; let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum(); diff --git a/datafusion/physical-plan/src/coop.rs b/datafusion/physical-plan/src/coop.rs index d929c78850e56..e340e37ed5075 100644 --- a/datafusion/physical-plan/src/coop.rs +++ b/datafusion/physical-plan/src/coop.rs @@ -80,6 +80,8 @@ use crate::filter_pushdown::{ FilterPushdownPropagation, }; use crate::projection::ProjectionExec; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, RecordBatchStream, SendableRecordBatchStream, @@ -282,8 +284,14 @@ impl ExecutionPlan for CooperativeExec { &self, partition: usize, task_ctx: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { - let child_stream = self.input.execute(partition, task_ctx)?; + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::execute_input; + + let child_stream = execute_input!(0, self.input, partition, task_ctx, state)?; Ok(make_cooperative(child_stream)) } diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index 52c37a106b39e..65b43209eb5f4 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -19,17 +19,22 @@ //! [`crate::displayable`] for examples of how to format use std::collections::{BTreeMap, HashMap}; -use std::fmt; use std::fmt::Formatter; +use std::fmt::{self, Debug}; +#[cfg(feature = "stateless_plan")] +use std::sync::Arc; use arrow::datatypes::SchemaRef; use datafusion_common::display::{GraphvizBuilder, PlanType, StringifiedPlan}; +use datafusion_execution::metrics::MetricsSet; use datafusion_expr::display_schema; use datafusion_physical_expr::LexOrdering; use crate::metrics::MetricType; use crate::render_tree::RenderTree; +#[cfg(feature = "stateless_plan")] +use crate::state::{ExecutionPlanStateVisitor, PlanStateNode, accept_state}; use super::{ExecutionPlan, ExecutionPlanVisitor, accept}; @@ -148,10 +153,19 @@ impl<'a> DisplayableExecutionPlan<'a> { /// Create a wrapper around an [`ExecutionPlan`] which can be /// pretty printed in a variety of ways that also shows aggregated /// metrics - pub fn with_metrics(inner: &'a dyn ExecutionPlan) -> Self { + pub fn with_metrics( + inner: &'a dyn ExecutionPlan, + #[cfg(feature = "stateless_plan")] state: Arc, + ) -> Self { + #[cfg(feature = "stateless_plan")] + let show_metrics = ShowMetrics::Aggregated { state }; + + #[cfg(not(feature = "stateless_plan"))] + let show_metrics = ShowMetrics::Aggregated; + Self { inner, - show_metrics: ShowMetrics::Aggregated, + show_metrics, show_statistics: false, show_schema: false, metric_types: Self::default_metric_types(), @@ -162,10 +176,19 @@ impl<'a> DisplayableExecutionPlan<'a> { /// Create a wrapper around an [`ExecutionPlan`] which can be /// pretty printed in a variety of ways that also shows all low /// level metrics - pub fn with_full_metrics(inner: &'a dyn ExecutionPlan) -> Self { + pub fn with_full_metrics( + inner: &'a dyn ExecutionPlan, + #[cfg(feature = "stateless_plan")] state: Arc, + ) -> Self { + #[cfg(feature = "stateless_plan")] + let show_metrics = ShowMetrics::Full { state }; + + #[cfg(not(feature = "stateless_plan"))] + let show_metrics = ShowMetrics::Full; + Self { inner, - show_metrics: ShowMetrics::Full, + show_metrics, show_statistics: false, show_schema: false, metric_types: Self::default_metric_types(), @@ -216,32 +239,39 @@ impl<'a> DisplayableExecutionPlan<'a> { } else { DisplayFormatType::Default }; + struct Wrapper<'a> { format_type: DisplayFormatType, - plan: &'a dyn ExecutionPlan, show_metrics: ShowMetrics, + plan: &'a dyn ExecutionPlan, show_statistics: bool, show_schema: bool, metric_types: Vec, } + impl fmt::Display for Wrapper<'_> { fn fmt(&self, f: &mut Formatter) -> fmt::Result { - let mut visitor = IndentVisitor { + let mut formatter = IndentFormatter { t: self.format_type, f, indent: 0, - show_metrics: self.show_metrics, show_statistics: self.show_statistics, show_schema: self.show_schema, - metric_types: &self.metric_types, }; - accept(self.plan, &mut visitor) + dispatch_formatter( + self.plan, + &self.show_metrics, + &self.metric_types, + &mut formatter, + false, // visit_only_root + ) } } + Wrapper { format_type, plan: self.inner, - show_metrics: self.show_metrics, + show_metrics: self.show_metrics.clone(), show_statistics: self.show_statistics, show_schema: self.show_schema, metric_types: self.metric_types.clone(), @@ -270,28 +300,30 @@ impl<'a> DisplayableExecutionPlan<'a> { fn fmt(&self, f: &mut Formatter) -> fmt::Result { let t = DisplayFormatType::Default; - let mut visitor = GraphvizVisitor { + let mut formatter = GraphvizFormatter { f, t, - show_metrics: self.show_metrics, show_statistics: self.show_statistics, - metric_types: &self.metric_types, graphviz_builder: GraphvizBuilder::default(), parents: Vec::new(), }; - visitor.start_graph()?; - - accept(self.plan, &mut visitor)?; - - visitor.end_graph()?; + formatter.start_graph()?; + dispatch_formatter( + self.plan, + &self.show_metrics, + &self.metric_types, + &mut formatter, + false, // visit_only_root + )?; + formatter.end_graph()?; Ok(()) } } Wrapper { plan: self.inner, - show_metrics: self.show_metrics, + show_metrics: self.show_metrics.clone(), show_statistics: self.show_statistics, metric_types: self.metric_types.clone(), } @@ -333,23 +365,26 @@ impl<'a> DisplayableExecutionPlan<'a> { impl fmt::Display for Wrapper<'_> { fn fmt(&self, f: &mut Formatter) -> fmt::Result { - let mut visitor = IndentVisitor { + let mut formatter = IndentFormatter { f, t: DisplayFormatType::Default, indent: 0, - show_metrics: self.show_metrics, show_statistics: self.show_statistics, show_schema: self.show_schema, - metric_types: &self.metric_types, }; - visitor.pre_visit(self.plan)?; - Ok(()) + dispatch_formatter( + self.plan, + &self.show_metrics, + &self.metric_types, + &mut formatter, + true, // visit_only_root + ) } } Wrapper { plan: self.inner, - show_metrics: self.show_metrics, + show_metrics: self.show_metrics.clone(), show_statistics: self.show_statistics, show_schema: self.show_schema, metric_types: self.metric_types.clone(), @@ -373,18 +408,215 @@ impl<'a> DisplayableExecutionPlan<'a> { } /// Enum representing the different levels of metrics to display -#[derive(Debug, Clone, Copy)] +#[derive(Clone)] enum ShowMetrics { /// Do not show any metrics None, /// Show aggregated metrics across partition + #[cfg(feature = "stateless_plan")] + Aggregated { state: Arc }, + + #[cfg(not(feature = "stateless_plan"))] Aggregated, /// Show full per-partition metrics + #[cfg(feature = "stateless_plan")] + Full { state: Arc }, + + #[cfg(not(feature = "stateless_plan"))] Full, } +impl Debug for ShowMetrics { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!( + f, + "{}", + match self { + Self::None => "none", + Self::Aggregated { .. } => "aggregated", + Self::Full { .. } => "full", + } + ) + } +} + +/// Helper trait to implement various plan display formats. +trait PlanFormatter { + fn pre_visit( + &mut self, + _plan: &dyn ExecutionPlan, + _metrics: Option, + ) -> Result; + + fn post_visit(&mut self, _plan: &dyn ExecutionPlan) -> Result; +} + +/// Helps to dispatch formatter depending on metrics requirements. +fn dispatch_formatter( + plan: &dyn ExecutionPlan, + show_metrics: &ShowMetrics, + metric_types: &[MetricType], + formatter: &mut impl PlanFormatter, + visit_only_root: bool, +) -> fmt::Result { + macro_rules! dispatch { + ($plan: expr, $visitor: expr) => { + if !visit_only_root { + accept($plan, &mut $visitor) + } else { + $visitor.pre_visit(plan).map(|_| ()) + } + }; + } + + #[cfg(feature = "stateless_plan")] + macro_rules! dispatch_with_state { + ($state: expr, $visitor: expr) => { + if !visit_only_root { + accept_state($state, &mut $visitor).map(|_| ()) + } else { + $visitor.pre_visit($state).map(|_| ()) + } + }; + } + + match &show_metrics { + ShowMetrics::None => { + // Metrics are not required, hence state is not required. + let mut visitor = PlanFormatterVisitor { + show_metrics: false, + formatter, + #[cfg(not(feature = "stateless_plan"))] + is_aggregated: false, + #[cfg(not(feature = "stateless_plan"))] + metric_types: &[], + }; + dispatch!(plan, visitor) + } + #[cfg(not(feature = "stateless_plan"))] + ShowMetrics::Aggregated | ShowMetrics::Full => { + let is_aggregated = matches!(show_metrics, ShowMetrics::Aggregated); + + // Take metrics from the plan itself. + let mut visitor = PlanFormatterVisitor { + show_metrics: true, + formatter, + is_aggregated, + metric_types, + }; + dispatch!(plan, visitor) + } + #[cfg(feature = "stateless_plan")] + ShowMetrics::Aggregated { state } | ShowMetrics::Full { state } => { + let is_aggregated = matches!(show_metrics, ShowMetrics::Aggregated { .. }); + + // Take metrics from the state. + let mut visitor = PlanWithStateFormatterVisitor { + formatter, + is_aggregated, + metric_types, + }; + dispatch_with_state!(state, visitor) + } + } +} + +/// Formats plan without state. +struct PlanFormatterVisitor<'a, T> { + formatter: &'a mut T, + show_metrics: bool, + #[cfg(not(feature = "stateless_plan"))] + is_aggregated: bool, + #[cfg(not(feature = "stateless_plan"))] + metric_types: &'a [MetricType], +} + +impl ExecutionPlanVisitor for PlanFormatterVisitor<'_, T> +where + T: PlanFormatter, +{ + type Error = fmt::Error; + + #[cfg(feature = "stateless_plan")] + fn pre_visit(&mut self, plan: &dyn ExecutionPlan) -> Result { + let _ = self.show_metrics; + self.formatter.pre_visit(plan, None) + } + + #[cfg(not(feature = "stateless_plan"))] + fn pre_visit(&mut self, plan: &dyn ExecutionPlan) -> Result { + let metrics = if !self.show_metrics { + None + } else { + plan.metrics().map(|metrics| { + if self.is_aggregated { + metrics + .filter_by_metric_types(self.metric_types) + .aggregate_by_name() + .sorted_for_display() + .timestamps_removed() + } else { + metrics.filter_by_metric_types(self.metric_types) + } + }) + }; + + self.formatter.pre_visit(plan, metrics) + } + + fn post_visit(&mut self, plan: &dyn ExecutionPlan) -> Result { + self.formatter.post_visit(plan) + } +} + +#[cfg(feature = "stateless_plan")] +mod visitor { + use crate::state::ExecutionPlanStateVisitor; + + use super::*; + + /// Formats plan with metrics. State is required. + pub struct PlanWithStateFormatterVisitor<'a, T> { + pub formatter: &'a mut T, + pub is_aggregated: bool, + pub metric_types: &'a [MetricType], + } + + impl ExecutionPlanStateVisitor for PlanWithStateFormatterVisitor<'_, T> + where + T: PlanFormatter, + { + type Error = fmt::Error; + + fn pre_visit(&mut self, state: &Arc) -> Result { + let metrics = state.metrics.clone_inner(); + let metrics = if self.is_aggregated { + metrics + .filter_by_metric_types(self.metric_types) + .aggregate_by_name() + .sorted_for_display() + .timestamps_removed() + } else { + metrics.filter_by_metric_types(self.metric_types) + }; + self.formatter + .pre_visit(state.plan_node.as_ref(), Some(metrics)) + } + + fn post_visit( + &mut self, + state: &Arc, + ) -> Result { + self.formatter.post_visit(state.plan_node.as_ref()) + } + } +} + +#[cfg(feature = "stateless_plan")] +use visitor::PlanWithStateFormatterVisitor; + /// Formats plans with a single line per node. /// /// # Example @@ -394,51 +626,33 @@ enum ShowMetrics { /// FilterExec: column1@0 = 5 /// ValuesExec /// ``` -struct IndentVisitor<'a, 'b> { +struct IndentFormatter<'a, 'b> { /// How to format each node t: DisplayFormatType, /// Write to this formatter f: &'a mut Formatter<'b>, /// Indent size indent: usize, - /// How to show metrics - show_metrics: ShowMetrics, /// If statistics should be displayed show_statistics: bool, /// If schema should be displayed show_schema: bool, - /// Which metric types should be rendered - metric_types: &'a [MetricType], } -impl ExecutionPlanVisitor for IndentVisitor<'_, '_> { - type Error = fmt::Error; - fn pre_visit(&mut self, plan: &dyn ExecutionPlan) -> Result { +impl PlanFormatter for IndentFormatter<'_, '_> { + fn pre_visit( + &mut self, + plan: &dyn ExecutionPlan, + metrics: Option, + ) -> Result { write!(self.f, "{:indent$}", "", indent = self.indent * 2)?; plan.fmt_as(self.t, self.f)?; - match self.show_metrics { - ShowMetrics::None => {} - ShowMetrics::Aggregated => { - if let Some(metrics) = plan.metrics() { - let metrics = metrics - .filter_by_metric_types(self.metric_types) - .aggregate_by_name() - .sorted_for_display() - .timestamps_removed(); - - write!(self.f, ", metrics=[{metrics}]")?; - } else { - write!(self.f, ", metrics=[]")?; - } - } - ShowMetrics::Full => { - if let Some(metrics) = plan.metrics() { - let metrics = metrics.filter_by_metric_types(self.metric_types); - write!(self.f, ", metrics=[{metrics}]")?; - } else { - write!(self.f, ", metrics=[]")?; - } - } + if let Some(metrics) = metrics { + if metrics.is_empty() { + write!(self.f, ", metrics=[]")? + } else { + write!(self.f, ", metrics=[{metrics}]")? + }; } if self.show_statistics { let stats = plan.partition_statistics(None).map_err(|_e| fmt::Error)?; @@ -456,29 +670,24 @@ impl ExecutionPlanVisitor for IndentVisitor<'_, '_> { Ok(true) } - fn post_visit(&mut self, _plan: &dyn ExecutionPlan) -> Result { + fn post_visit(&mut self, _plan: &dyn ExecutionPlan) -> Result { self.indent -= 1; Ok(true) } } -struct GraphvizVisitor<'a, 'b> { +struct GraphvizFormatter<'a, 'b> { f: &'a mut Formatter<'b>, /// How to format each node t: DisplayFormatType, - /// How to show metrics - show_metrics: ShowMetrics, /// If statistics should be displayed show_statistics: bool, - /// Which metric types should be rendered - metric_types: &'a [MetricType], - graphviz_builder: GraphvizBuilder, /// Used to record parent node ids when visiting a plan. parents: Vec, } -impl GraphvizVisitor<'_, '_> { +impl GraphvizFormatter<'_, '_> { fn start_graph(&mut self) -> fmt::Result { self.graphviz_builder.start_graph(self.f) } @@ -488,10 +697,12 @@ impl GraphvizVisitor<'_, '_> { } } -impl ExecutionPlanVisitor for GraphvizVisitor<'_, '_> { - type Error = fmt::Error; - - fn pre_visit(&mut self, plan: &dyn ExecutionPlan) -> Result { +impl PlanFormatter for GraphvizFormatter<'_, '_> { + fn pre_visit( + &mut self, + plan: &dyn ExecutionPlan, + metrics: Option, + ) -> Result { let id = self.graphviz_builder.next_id(); struct Wrapper<'a>(&'a dyn ExecutionPlan, DisplayFormatType); @@ -503,30 +714,15 @@ impl ExecutionPlanVisitor for GraphvizVisitor<'_, '_> { } let label = { format!("{}", Wrapper(plan, self.t)) }; - - let metrics = match self.show_metrics { - ShowMetrics::None => "".to_string(), - ShowMetrics::Aggregated => { - if let Some(metrics) = plan.metrics() { - let metrics = metrics - .filter_by_metric_types(self.metric_types) - .aggregate_by_name() - .sorted_for_display() - .timestamps_removed(); - - format!("metrics=[{metrics}]") - } else { + let metrics = match metrics { + Some(metrics) => { + if metrics.is_empty() { "metrics=[]".to_string() - } - } - ShowMetrics::Full => { - if let Some(metrics) = plan.metrics() { - let metrics = metrics.filter_by_metric_types(self.metric_types); - format!("metrics=[{metrics}]") } else { - "metrics=[]".to_string() + format!("metrics=[{metrics}]") } } + None => "".to_string(), }; let statistics = if self.show_statistics { @@ -559,7 +755,7 @@ impl ExecutionPlanVisitor for GraphvizVisitor<'_, '_> { Ok(true) } - fn post_visit(&mut self, _plan: &dyn ExecutionPlan) -> Result { + fn post_visit(&mut self, _plan: &dyn ExecutionPlan) -> Result { self.parents.pop(); Ok(true) } @@ -1123,6 +1319,8 @@ mod tests { use datafusion_common::{Result, Statistics, internal_datafusion_err}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; + #[cfg(feature = "stateless_plan")] + use crate::state::PlanStateNode; use crate::{DisplayAs, ExecutionPlan, PlanProperties}; use super::DisplayableExecutionPlan; @@ -1172,6 +1370,7 @@ mod tests { &self, _: usize, _: Arc, + #[cfg(feature = "stateless_plan")] _: &Arc, ) -> Result { todo!() } diff --git a/datafusion/physical-plan/src/dynamic_filter.rs b/datafusion/physical-plan/src/dynamic_filter.rs new file mode 100644 index 0000000000000..9c6d6acd48608 --- /dev/null +++ b/datafusion/physical-plan/src/dynamic_filter.rs @@ -0,0 +1,88 @@ +// 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. + +//! Utilities for working with dynamic filters in both stateful and stateless plan modes. +//! +//! Each dynamic filter can exist in two states: planning and executable.The difference +//! between these two states is that a planned filter does not assume concurrent modifications, +//! while an executable filter does. During the planning or optimization stage, a filter is created +//! as a planned filter and then converted into an executable form when [`execute`] is called. +//! +//! For stateful (default) plan mode, there is no need to distinguish between planned +//! and executable filters, as they are stored directly within the owner's [`ExecutionPlan`] +//! and shared with children during the planning stage via filter push-down optimization. +//! In this mode, both states are represented by the same type: [`DynamicFilterPhysicalExpr`]. +//! +//! For stateless plan mode, filters are similarly pushed from the owner's [`ExecutionPlan`] +//! to a child during filter push-down. However, because the [`ExecutionPlan`] is stateless, +//! it cannot store a shared version of the filter. Instead, the executable filter is created +//! when [`execute`] is called. In this mode, the two states are represented by different types: +//! the planned version is [`PlannedDynamicFilterPhysicalExpr`] from the physical-expr crate, +//! and the executable version remains [`DynamicFilterPhysicalExpr`]. +//! +//! [`ExecutionPlan`]: crate::ExecutionPlan +//! [`execute`]: crate::ExecutionPlan::execute +//! + +use std::sync::Arc; + +use datafusion_physical_expr::PhysicalExpr; + +pub use datafusion_physical_expr::expressions::DynamicFilterPhysicalExpr; + +#[cfg(feature = "stateless_plan")] +pub use datafusion_physical_expr::expressions::PlannedDynamicFilterPhysicalExpr; + +/// For stateful plans planned and executable filters are the same. +#[cfg(not(feature = "stateless_plan"))] +pub type PlannedDynamicFilterPhysicalExpr = DynamicFilterPhysicalExpr; + +/// Helper to make a new planning stage dynamic filter. +#[cfg(feature = "stateless_plan")] +pub fn make_planned_dynamic_filter( + expr: Arc, + children: Vec>, +) -> PlannedDynamicFilterPhysicalExpr { + PlannedDynamicFilterPhysicalExpr::new(expr, children) +} + +#[cfg(not(feature = "stateless_plan"))] +pub fn make_planned_dynamic_filter( + expr: Arc, + children: Vec>, +) -> PlannedDynamicFilterPhysicalExpr { + // For stateful plans executable planned and executable filters are the same. + make_executable_dynamic_filter(expr, children) +} + +/// Helper to make a new execution stage [`DynamicFilterPhysicalExpr`]. +#[cfg(feature = "stateless_plan")] +#[cfg(test)] +pub fn make_executable_dynamic_filter( + expr: Arc, + children: Vec>, +) -> DynamicFilterPhysicalExpr { + PlannedDynamicFilterPhysicalExpr::new(expr, children).to_executable() +} + +#[cfg(not(feature = "stateless_plan"))] +pub fn make_executable_dynamic_filter( + expr: Arc, + children: Vec>, +) -> DynamicFilterPhysicalExpr { + DynamicFilterPhysicalExpr::new(expr, children) +} diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index fcfbcfa3e8277..40a84e5bf22f7 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -21,6 +21,8 @@ use std::any::Any; use std::sync::Arc; use crate::memory::MemoryStream; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics, common}; use crate::{ DisplayFormatType, ExecutionPlan, Partitioning, @@ -133,6 +135,7 @@ impl ExecutionPlan for EmptyExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { trace!( "Start EmptyExec::execute for partition {} of context session_id {} and task_id {:?}", @@ -183,6 +186,7 @@ impl ExecutionPlan for EmptyExec { #[cfg(test)] mod tests { use super::*; + use crate::execution_plan::execute_plan; use crate::test; use crate::with_new_children_if_necessary; @@ -195,7 +199,7 @@ mod tests { assert_eq!(empty.schema(), schema); // We should have no results - let iter = empty.execute(0, task_ctx)?; + let iter = execute_plan(Arc::new(empty), 0, task_ctx)?; let batches = common::collect(iter).await?; assert!(batches.is_empty()); @@ -225,11 +229,11 @@ mod tests { async fn invalid_execute() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let schema = test::aggr_test_schema(); - let empty = EmptyExec::new(schema); + let empty: Arc = Arc::new(EmptyExec::new(schema)); // ask for the wrong partition - assert!(empty.execute(1, Arc::clone(&task_ctx)).is_err()); - assert!(empty.execute(20, task_ctx).is_err()); + assert!(execute_plan(Arc::clone(&empty), 1, Arc::clone(&task_ctx)).is_err()); + assert!(execute_plan(Arc::clone(&empty), 20, Arc::clone(&task_ctx)).is_err()); Ok(()) } } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 06da0b8933c18..64800cbc58d35 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -24,16 +24,23 @@ pub use crate::metrics::Metric; pub use crate::ordering::InputOrderMode; use crate::sort_pushdown::SortOrderPushdownResult; pub use crate::stream::EmptyRecordBatchStream; +#[cfg(not(feature = "stateless_plan"))] +use crate::stream::RecordBatchStreamAdapter; +#[cfg(feature = "stateless_plan")] +use {crate::state::PlanStateNode, crate::state::WithPlanStateNode}; pub use datafusion_common::hash_utils; pub use datafusion_common::utils::project_schema; pub use datafusion_common::{ColumnStatistics, Statistics, internal_err}; +use datafusion_execution::metrics::MetricsSet; pub use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; pub use datafusion_expr::{Accumulator, ColumnarValue}; pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ Distribution, Partitioning, PhysicalExpr, expressions, }; +#[cfg(not(feature = "stateless_plan"))] +use futures::StreamExt; use std::any::Any; use std::fmt::Debug; @@ -41,9 +48,7 @@ use std::sync::Arc; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::display::DisplayableExecutionPlan; -use crate::metrics::MetricsSet; use crate::projection::ProjectionExec; -use crate::stream::RecordBatchStreamAdapter; use arrow::array::{Array, RecordBatch}; use arrow::datatypes::SchemaRef; @@ -59,7 +64,7 @@ use datafusion_physical_expr_common::sort_expr::{ LexOrdering, OrderingRequirements, PhysicalSortExpr, }; -use futures::stream::{StreamExt, TryStreamExt}; +use futures::stream::TryStreamExt; /// Represent nodes in the DataFusion Physical Plan. /// @@ -81,6 +86,18 @@ use futures::stream::{StreamExt, TryStreamExt}; /// [`required_input_distribution`]: ExecutionPlan::required_input_distribution /// [`required_input_ordering`]: ExecutionPlan::required_input_ordering /// +/// # State +/// +/// Two modes are supported for physical plan states: +/// +/// * Default mode: a plan state, which must be shared across partitions or +/// different nodes during execution, is stored directly within each plan. +/// In this mode, a plan generally cannot be reused. +/// +/// * Statelss mode: a state is stored in a separate state tree created each +/// time the plan is executed. In this mode, a plan can be reused and executed +/// concurrently using separate state trees. +/// /// # Examples /// /// See [`datafusion-examples`] for examples, including @@ -230,6 +247,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// thus it is expected that any cached plan properties will remain valid after the reset. /// /// [`DynamicFilterPhysicalExpr`]: datafusion_physical_expr::expressions::DynamicFilterPhysicalExpr + #[cfg(not(feature = "stateless_plan"))] fn reset_state(self: Arc) -> Result> { let children = self.children().into_iter().cloned().collect(); self.with_new_children(children) @@ -321,6 +339,17 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// The [cancellation benchmark] tracks some cases of how quickly queries can /// be cancelled. /// + /// # State + /// + /// In stateless plan mode, the provided `state` must be preserved for the duration + /// of plan execution to guarantee correct results. For non-root nodes, this is + /// ensured by the [`Arc`]s stored within the parent state node, so it is important + /// to maintain this invariant for the root node. + /// + /// Helper functions are available to simplify working with state. For example, + /// [`execute_plan`] and [`execute_stream`] embed the state root into the returned + /// stream, ensuring it is not dropped until the stream itself is dropped. + /// /// For more details see [`SpawnedTask`], [`JoinSet`] and [`RecordBatchReceiverStreamBuilder`] /// for structures to help ensure all background tasks are cancelled. /// @@ -454,6 +483,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result; /// Return a snapshot of the set of [`Metric`]s for this @@ -467,6 +497,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// resolved) for all available partitions, the set of metrics /// should be complete. If this function is called prior to /// `execute()` new metrics may appear in subsequent calls. + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { None } @@ -679,6 +710,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// in order to wire up the working table used during recursive-CTE execution. /// Similar patterns can be followed by custom nodes that need late-bound /// dependencies or shared state. + #[cfg(not(feature = "stateless_plan"))] fn with_new_state( &self, _state: Arc, @@ -1182,7 +1214,8 @@ pub fn displayable(plan: &dyn ExecutionPlan) -> DisplayableExecutionPlan<'_> { DisplayableExecutionPlan::new(plan) } -/// Execute the [ExecutionPlan] and collect the results in memory +/// Execute the [ExecutionPlan] and collect the results in memory. +#[cfg(not(feature = "stateless_plan"))] pub async fn collect( plan: Arc, context: Arc, @@ -1191,6 +1224,60 @@ pub async fn collect( crate::common::collect(stream).await } +/// Execute the [ExecutionPlan] and collect the results in memory. +#[cfg(feature = "stateless_plan")] +pub async fn collect( + plan: Arc, + context: Arc, +) -> Result> { + collect_preserving_state(plan, context) + .await + .as_result() + .map(|r| r.into_inner()) +} + +/// Execute the [ExecutionPlan] and collect the results in memory, +/// returning passed `metrics_plan` acquired just after plan execution. +#[cfg(not(feature = "stateless_plan"))] +pub async fn collect_and_get_metrics_of( + plan: Arc, + metrics_plan: &Arc, + context: Arc, +) -> Result<(Vec, Option)> { + let batches = collect(Arc::clone(&plan), context).await?; + Ok((batches, metrics_plan.metrics())) +} + +/// Execute the [ExecutionPlan] and collect the results in memory, +/// returning passed `metrics_plan` acquired just after plan execution. +#[cfg(feature = "stateless_plan")] +pub async fn collect_and_get_metrics_of( + plan: Arc, + metrics_plan: &Arc, + context: Arc, +) -> Result<(Vec, Option)> { + let (batches, state) = collect_preserving_state(plan, context) + .await + .as_result()? + .into_parts(); + let metrics = state.metrics_of(metrics_plan); + Ok((batches, metrics)) +} + +/// Executes the [`ExecutionPlan`] and collects the results in memory. +/// This function also preserves the plan state and returns it to the caller, +/// which can be useful, for example, for accessing node metrics. +#[cfg(feature = "stateless_plan")] +pub async fn collect_preserving_state( + plan: Arc, + context: Arc, +) -> WithPlanStateNode>> { + execute_stream_preserving_state(plan, context) + .try_map_async(|stream| async move { Ok(crate::common::collect(stream?).await) }) + .await + .unwrap() +} + /// Execute the [ExecutionPlan] and return a single stream of `RecordBatch`es. /// /// See [collect] to buffer the `RecordBatch`es in memory. @@ -1203,16 +1290,18 @@ pub async fn collect( clippy::needless_pass_by_value, reason = "Public API that historically takes owned Arcs" )] +#[cfg(not(feature = "stateless_plan"))] pub fn execute_stream( plan: Arc, context: Arc, ) -> Result { match plan.output_partitioning().partition_count() { - 0 => Ok(Box::pin(EmptyRecordBatchStream::new(plan.schema()))), + 0 => Ok(Box::pin(EmptyRecordBatchStream::new(plan.schema())) + as SendableRecordBatchStream), 1 => plan.execute(0, context), 2.. => { // merge into a single partition - let plan = CoalescePartitionsExec::new(Arc::clone(&plan)); + let plan = Arc::new(CoalescePartitionsExec::new(Arc::clone(&plan))); // CoalescePartitionsExec must produce a single partition assert_eq!(1, plan.properties().output_partitioning().partition_count()); plan.execute(0, context) @@ -1220,6 +1309,122 @@ pub fn execute_stream( } } +/// Execute the [ExecutionPlan] and return a single stream of `RecordBatch`es. +/// +/// See [collect] to buffer the `RecordBatch`es in memory. +/// +/// # Aborting Execution +/// +/// Dropping the stream will abort the execution of the query, and free up +/// any allocated resources +#[cfg(feature = "stateless_plan")] +pub fn execute_stream( + plan: Arc, + context: Arc, +) -> Result { + let stream_with_state = execute_stream_preserving_state(plan, context).as_result()?; + Ok(Box::pin(stream_with_state)) +} + +#[cfg(feature = "stateless_plan")] +pub fn execute_stream_and_get_metrics_of( + plan: Arc, + metrics_plan: &Arc, + context: Arc, +) -> Result<(SendableRecordBatchStream, Option)> { + let stream_with_state = execute_stream_preserving_state(plan, context).as_result()?; + let metrics = stream_with_state.state().metrics_of(metrics_plan); + Ok((Box::pin(stream_with_state), metrics)) +} + +/// Execute the [ExecutionPlan]. +#[cfg(feature = "stateless_plan")] +pub fn execute_stream_preserving_state( + plan: Arc, + context: Arc, +) -> WithPlanStateNode> { + let num_partitions = plan.output_partitioning().partition_count(); + if num_partitions == 0 { + let plan_state = PlanStateNode::new_root_arc(Arc::clone(&plan)); + let stream = Box::pin(EmptyRecordBatchStream::new(plan.schema())); + return WithPlanStateNode::new(Ok(stream), plan_state); + } + + let plan = if num_partitions == 1 { + plan + } else { + Arc::new(CoalescePartitionsExec::new(plan)) + }; + + execute_plan_preserving_state(plan, 0, context) +} + +/// Execute a particular partition of the [ExecutionPlan]. +/// This function also preserves the plan state and returns it to the caller, +/// which can be useful, for example, for accessing node metrics. +#[expect( + clippy::needless_pass_by_value, + reason = "consistent with `execute_stream(...)`" +)] +#[cfg(feature = "stateless_plan")] +pub fn execute_plan_preserving_state( + plan: Arc, + partition: usize, + context: Arc, +) -> WithPlanStateNode> { + let plan_state = PlanStateNode::new_root_arc(Arc::clone(&plan)); + let stream_res = plan.execute(partition, context, &plan_state); + WithPlanStateNode::new(stream_res, plan_state) +} + +#[cfg(feature = "stateless_plan")] +pub fn execute_plan( + plan: Arc, + partition: usize, + context: Arc, +) -> Result { + let stream_with_state = + execute_plan_preserving_state(plan, partition, context).as_result()?; + Ok(Box::pin(stream_with_state)) +} + +#[cfg(not(feature = "stateless_plan"))] +#[expect( + clippy::needless_pass_by_value, + reason = "consistent with `execute_stream(...)`" +)] +pub fn execute_plan( + plan: Arc, + partition: usize, + context: Arc, +) -> Result { + plan.execute(partition, context) +} + +#[cfg(not(feature = "stateless_plan"))] +pub fn execute_plan_and_get_metrics_of( + plan: Arc, + metrics_plan: &Arc, + partition: usize, + context: Arc, +) -> Result<(SendableRecordBatchStream, Option)> { + let stream = execute_plan(plan, partition, context)?; + Ok((stream, metrics_plan.metrics())) +} + +#[cfg(feature = "stateless_plan")] +pub fn execute_plan_and_get_metrics_of( + plan: Arc, + metrics_plan: &Arc, + partition: usize, + context: Arc, +) -> Result<(SendableRecordBatchStream, Option)> { + let stream_with_state = + execute_plan_preserving_state(plan, partition, context).as_result()?; + let metrics = stream_with_state.state().metrics_of(metrics_plan); + Ok((Box::pin(stream_with_state), metrics)) +} + /// Execute the [ExecutionPlan] and collect the results in memory pub async fn collect_partitioned( plan: Arc, @@ -1271,6 +1476,7 @@ pub async fn collect_partitioned( clippy::needless_pass_by_value, reason = "Public API that historically takes owned Arcs" )] +#[cfg(not(feature = "stateless_plan"))] pub fn execute_stream_partitioned( plan: Arc, context: Arc, @@ -1283,6 +1489,24 @@ pub fn execute_stream_partitioned( Ok(streams) } +#[expect( + clippy::needless_pass_by_value, + reason = "Public API that historically takes owned Arcs" +)] +#[cfg(feature = "stateless_plan")] +pub fn execute_stream_partitioned( + plan: Arc, + context: Arc, +) -> Result> { + let num_partitions = plan.output_partitioning().partition_count(); + let mut streams = Vec::with_capacity(num_partitions); + let plan_state = PlanStateNode::new_root_arc(Arc::clone(&plan)); + for i in 0..num_partitions { + streams.push(plan.execute(i, Arc::clone(&context), &plan_state)?); + } + Ok(streams) +} + /// Executes an input stream and ensures that the resulting stream adheres to /// the `not null` constraints specified in the `sink_schema`. /// @@ -1306,6 +1530,7 @@ pub fn execute_stream_partitioned( clippy::needless_pass_by_value, reason = "Public API that historically takes owned Arcs" )] +#[cfg(not(feature = "stateless_plan"))] pub fn execute_input_stream( input: Arc, sink_schema: SchemaRef, @@ -1331,6 +1556,7 @@ pub fn execute_input_stream( Ok(input_stream) } else { // Check not null constraint on the input stream + Ok(Box::pin(RecordBatchStreamAdapter::new( sink_schema, input_stream @@ -1405,6 +1631,39 @@ pub enum CardinalityEffect { GreaterEqual, } +// Macros helping to unify a work with the plan for both stateless/stateful method. + +/// Helps to acquire metrics. +#[macro_export] +macro_rules! plan_metrics { + ($self: expr, $state: expr $(,)?) => {{ + #[cfg(feature = "stateless_plan")] + { + &$state.metrics + } + #[cfg(not(feature = "stateless_plan"))] + { + &$self.metrics + } + }}; +} + +/// Helps to execute a child plan. +#[macro_export] +macro_rules! execute_input { + ($child_idx: expr, $input: expr, $partition: expr, $task_ctx: expr, $state: expr $(,)?) => {{ + #[cfg(feature = "stateless_plan")] + { + $input.execute($partition, $task_ctx, &$state.child_state($child_idx)) + } + + #[cfg(not(feature = "stateless_plan"))] + { + $input.execute($partition, $task_ctx) + } + }}; +} + #[cfg(test)] mod tests { use std::any::Any; @@ -1416,7 +1675,7 @@ mod tests { use arrow::array::{DictionaryArray, Int32Array, NullArray, RunArray}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{Result, Statistics}; - use datafusion_execution::{SendableRecordBatchStream, TaskContext}; + use datafusion_execution::SendableRecordBatchStream; #[derive(Debug)] pub struct EmptyExec; @@ -1465,6 +1724,7 @@ mod tests { &self, _partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { unimplemented!() } @@ -1532,6 +1792,7 @@ mod tests { &self, _partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { unimplemented!() } diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index aa3c0afefe8b5..d5ff869e930d7 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -22,6 +22,8 @@ use std::sync::Arc; use super::{DisplayAs, PlanProperties, SendableRecordBatchStream}; use crate::execution_plan::{Boundedness, EmissionType}; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; @@ -132,6 +134,7 @@ impl ExecutionPlan for ExplainExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { trace!( "Start ExplainExec::execute for partition {} of context session_id {} and task_id {:?}", diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 674fe6692adf5..d936bdcae9a70 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -39,9 +39,11 @@ use crate::projection::{ EmbeddedProjection, ProjectionExec, ProjectionExpr, make_with_child, try_embed_projection, update_expr, }; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayFormatType, ExecutionPlan, - metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet, RatioMetrics}, + metrics::{BaselineMetrics, ExecutionPlanMetricsSet, RatioMetrics}, }; use arrow::compute::filter_record_batch; @@ -54,6 +56,8 @@ use datafusion_common::{ DataFusionError, Result, ScalarValue, internal_err, plan_err, project_schema, }; use datafusion_execution::TaskContext; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::MetricsSet; use datafusion_expr::Operator; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::{BinaryExpr, Column, lit}; @@ -79,8 +83,6 @@ pub struct FilterExec { predicate: Arc, /// The input plan input: Arc, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// Selectivity for statistics. 0 = no rows, 100 = all rows default_selectivity: u8, /// Properties equivalence properties, partitioning, etc. @@ -91,6 +93,9 @@ pub struct FilterExec { batch_size: usize, /// Number of rows to fetch fetch: Option, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } impl FilterExec { @@ -112,12 +117,13 @@ impl FilterExec { Ok(Self { predicate, input: Arc::clone(&input), - metrics: ExecutionPlanMetricsSet::new(), default_selectivity, cache, projection: None, batch_size: FILTER_EXEC_DEFAULT_BATCH_SIZE, fetch: None, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), }) } other => { @@ -161,6 +167,7 @@ impl FilterExec { Ok(Self { predicate: Arc::clone(&self.predicate), input: Arc::clone(&self.input), + #[cfg(not(feature = "stateless_plan"))] metrics: self.metrics.clone(), default_selectivity: self.default_selectivity, cache, @@ -174,6 +181,7 @@ impl FilterExec { Ok(Self { predicate: Arc::clone(&self.predicate), input: Arc::clone(&self.input), + #[cfg(not(feature = "stateless_plan"))] metrics: self.metrics.clone(), default_selectivity: self.default_selectivity, cache: self.cache.clone(), @@ -413,18 +421,24 @@ impl ExecutionPlan for FilterExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + trace!( "Start FilterExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id() ); - let metrics = FilterExecMetrics::new(&self.metrics, partition); + let metrics = FilterExecMetrics::new(plan_metrics!(self, state), partition); Ok(Box::pin(FilterExecStream { schema: self.schema(), predicate: Arc::clone(&self.predicate), - input: self.input.execute(partition, context)?, + input: execute_input!(0, self.input, partition, context, state)?, metrics, projection: self.projection.clone(), batch_coalescer: LimitedBatchCoalescer::new( @@ -435,6 +449,7 @@ impl ExecutionPlan for FilterExec { })) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -582,6 +597,7 @@ impl ExecutionPlan for FilterExec { let new = FilterExec { predicate: Arc::clone(&new_predicate), input: Arc::clone(&filter_input), + #[cfg(not(feature = "stateless_plan"))] metrics: self.metrics.clone(), default_selectivity: self.default_selectivity, cache: Self::compute_properties( @@ -607,6 +623,7 @@ impl ExecutionPlan for FilterExec { Some(Arc::new(Self { predicate: Arc::clone(&self.predicate), input: Arc::clone(&self.input), + #[cfg(not(feature = "stateless_plan"))] metrics: self.metrics.clone(), default_selectivity: self.default_selectivity, cache: self.cache.clone(), diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 4f32b6176ec39..483648798f5af 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -26,11 +26,12 @@ use super::utils::{ reorder_output_after_swap, }; use crate::execution_plan::{EmissionType, boundedness_from_children}; -use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::projection::{ ProjectionExec, join_allows_pushdown, join_table_borders, new_join_children, physical_to_column_exprs, }; +#[cfg(feature = "stateless_plan")] +use crate::state::{PlanState, PlanStateNode}; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, PlanProperties, RecordBatchStream, @@ -46,6 +47,8 @@ use datafusion_common::{ }; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_expr::equivalence::join_equivalence_properties; use async_trait::async_trait; @@ -84,6 +87,17 @@ pub struct CrossJoinExec { pub right: Arc, /// The schema once the join is applied schema: SchemaRef, + /// Properties such as schema, equivalence properties, ordering, partitioning, etc. + cache: PlanProperties, + /// Execution plan metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, + #[cfg(not(feature = "stateless_plan"))] + state: CrossJoinExecState, +} + +#[derive(Debug, Default)] +struct CrossJoinExecState { /// Buffered copy of left (build) side in memory. /// /// This structure is *shared* across all output streams. @@ -91,10 +105,13 @@ pub struct CrossJoinExec { /// Each output stream waits on the `OnceAsync` to signal the completion of /// the left side loading. left_fut: OnceAsync, - /// Execution plan metrics - metrics: ExecutionPlanMetricsSet, - /// Properties such as schema, equivalence properties, ordering, partitioning, etc. - cache: PlanProperties, +} + +#[cfg(feature = "stateless_plan")] +impl PlanState for CrossJoinExecState { + fn as_any(&self) -> &dyn Any { + self + } } impl CrossJoinExec { @@ -123,9 +140,11 @@ impl CrossJoinExec { left, right, schema, - left_fut: Default::default(), - metrics: ExecutionPlanMetricsSet::default(), cache, + #[cfg(not(feature = "stateless_plan"))] + state: Default::default(), + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::default(), } } @@ -264,6 +283,7 @@ impl ExecutionPlan for CrossJoinExec { vec![&self.left, &self.right] } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -278,12 +298,13 @@ impl ExecutionPlan for CrossJoinExec { ))) } + #[cfg(not(feature = "stateless_plan"))] fn reset_state(self: Arc) -> Result> { let new_exec = CrossJoinExec { left: Arc::clone(&self.left), right: Arc::clone(&self.right), schema: Arc::clone(&self.schema), - left_fut: Default::default(), // reset the build side! + state: Default::default(), // reset the build side! metrics: ExecutionPlanMetricsSet::default(), cache: self.cache.clone(), }; @@ -301,7 +322,13 @@ impl ExecutionPlan for CrossJoinExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + assert_eq_or_internal_err!( self.left.output_partitioning().partition_count(), 1, @@ -309,9 +336,11 @@ impl ExecutionPlan for CrossJoinExec { consider using CoalescePartitionsExec or the EnforceDistribution rule" ); - let stream = self.right.execute(partition, Arc::clone(&context))?; + let stream = + execute_input!(1, self.right, partition, Arc::clone(&context), state)?; - let join_metrics = BuildProbeJoinMetrics::new(partition, &self.metrics); + let join_metrics = + BuildProbeJoinMetrics::new(partition, plan_metrics!(self, state)); // Initialization of operator-level reservation let reservation = @@ -321,8 +350,16 @@ impl ExecutionPlan for CrossJoinExec { let enforce_batch_size_in_joins = context.session_config().enforce_batch_size_in_joins(); - let left_fut = self.left_fut.try_once(|| { - let left_stream = self.left.execute(0, context)?; + #[cfg(feature = "stateless_plan")] + let left_fut = &state + .get_or_init_state(CrossJoinExecState::default) + .left_fut; + + #[cfg(not(feature = "stateless_plan"))] + let left_fut = &self.state.left_fut; + + let left_fut = left_fut.try_once(|| { + let left_stream = execute_input!(0, self.left, 0, context, state)?; Ok(load_left_input( left_stream, @@ -668,9 +705,11 @@ impl CrossJoinStream { mod tests { use super::*; use crate::common; + use crate::execution_plan::execute_plan_and_get_metrics_of; use crate::test::{assert_join_metrics, build_table_scan_i32}; use datafusion_common::{assert_contains, test_util::batches_to_sort_string}; + use datafusion_execution::metrics::MetricsSet; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use insta::assert_snapshot; @@ -679,14 +718,13 @@ mod tests { right: Arc, context: Arc, ) -> Result<(Vec, Vec, MetricsSet)> { - let join = CrossJoinExec::new(left, right); + let join: Arc = Arc::new(CrossJoinExec::new(left, right)); let columns_header = columns(&join.schema()); - let stream = join.execute(0, context)?; + let (stream, metrics) = + execute_plan_and_get_metrics_of(Arc::clone(&join), &join, 0, context)?; let batches = common::collect(stream).await?; - let metrics = join.metrics().unwrap(); - - Ok((columns_header, batches, metrics)) + Ok((columns_header, batches, metrics.unwrap())) } #[tokio::test] diff --git a/datafusion/physical-plan/src/joins/hash_join/exec.rs b/datafusion/physical-plan/src/joins/hash_join/exec.rs index bd92cf496426f..2577380ed0a46 100644 --- a/datafusion/physical-plan/src/joins/hash_join/exec.rs +++ b/datafusion/physical-plan/src/joins/hash_join/exec.rs @@ -22,6 +22,8 @@ use std::sync::{Arc, OnceLock}; use std::{any::Any, vec}; use crate::ExecutionPlanProperties; +use crate::dynamic_filter::PlannedDynamicFilterPhysicalExpr; +use crate::dynamic_filter::make_planned_dynamic_filter; use crate::execution_plan::{EmissionType, boundedness_from_children}; use crate::filter_pushdown::{ ChildPushdownResult, FilterDescription, FilterPushdownPhase, @@ -55,7 +57,11 @@ use crate::{ build_join_schema, check_join_is_valid, estimate_join_statistics, need_produce_result_in_final, symmetric_join_output_partitioning, }, - metrics::{ExecutionPlanMetricsSet, MetricsSet}, +}; +#[cfg(feature = "stateless_plan")] +use crate::{ + dynamic_filter::DynamicFilterPhysicalExpr, + state::{PlanState, PlanStateNode}, }; use arrow::array::{ArrayRef, BooleanBufferBuilder}; @@ -72,12 +78,14 @@ use datafusion_common::{ }; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_expr::Accumulator; use datafusion_functions_aggregate_common::min_max::{MaxAccumulator, MinAccumulator}; use datafusion_physical_expr::equivalence::{ ProjectionMapping, join_equivalence_properties, }; -use datafusion_physical_expr::expressions::{DynamicFilterPhysicalExpr, lit}; +use datafusion_physical_expr::expressions::lit; use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef}; use ahash::RandomState; @@ -329,19 +337,10 @@ pub struct HashJoinExec { /// The schema after join. Please be careful when using this schema, /// if there is a projection, the schema isn't the same as the output schema. join_schema: SchemaRef, - /// Future that consumes left input and builds the hash table - /// - /// For CollectLeft partition mode, this structure is *shared* across all output streams. - /// - /// Each output stream waits on the `OnceAsync` to signal the completion of - /// the hash table creation. - left_fut: Arc>, /// Shared the `SeededRandomState` for the hashing algorithm (seeds preserved for serialization) random_state: SeededRandomState, /// Partitioning mode to use pub mode: PartitionMode, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// The projection indices of the columns in the output schema of join pub projection: Option>, /// Information of index and left / right placement of columns @@ -350,21 +349,64 @@ pub struct HashJoinExec { pub null_equality: NullEquality, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, - /// Dynamic filter for pushing down to the probe side - /// Set when dynamic filter pushdown is detected in handle_child_pushdown_result. - /// HashJoinExec also needs to keep a shared bounds accumulator for coordinating updates. - dynamic_filter: Option, + /// Dynamic filter for pushing down to the probe side. + dynamic_filter: Option>, + /// Plan state. + #[cfg(not(feature = "stateless_plan"))] + state: HashJoinExecState, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } -#[derive(Clone)] -struct HashJoinExecDynamicFilter { - /// Dynamic filter that we'll update with the results of the build side once that is done. - filter: Arc, +#[derive(Default, Clone)] +struct HashJoinExecState { + /// Future that consumes left input and builds the hash table + /// + /// For CollectLeft partition mode, this structure is *shared* across all output streams. + /// + /// Each output stream waits on the `OnceAsync` to signal the completion of + /// the hash table creation. + left_fut: Arc>, + + /// Dynamic filter for pushing down to the probe side. + /// Note: it is originated from [`HashJoinExec::dynamic_filter`] on execution stage. + /// It is not stored in the case of stateful plan as in this case planned and executable + /// filters are the same and filter is stored directly in [`HashJoinExec`]. + #[cfg(feature = "stateless_plan")] + dynamic_filter: Option>, + /// Build accumulator to collect build-side information (hash maps and/or bounds) from each partition. /// It is lazily initialized during execution to make sure we use the actual execution time partition counts. build_accumulator: OnceLock>, } +impl HashJoinExecState { + #[cfg(feature = "stateless_plan")] + fn new(dynamic_filter: Option>) -> Self { + Self { + left_fut: Default::default(), + dynamic_filter, + build_accumulator: Default::default(), + } + } +} + +#[cfg(feature = "stateless_plan")] +impl PlanState for HashJoinExecState { + fn as_any(&self) -> &dyn Any { + self + } + + fn dynamic_filters(&self) -> Vec> { + if let Some(filter) = self.dynamic_filter.as_ref() { + vec![Arc::clone(filter)] + } else { + vec![] + } + } +} + impl fmt::Debug for HashJoinExec { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("HashJoinExec") @@ -374,10 +416,8 @@ impl fmt::Debug for HashJoinExec { .field("filter", &self.filter) .field("join_type", &self.join_type) .field("join_schema", &self.join_schema) - .field("left_fut", &self.left_fut) .field("random_state", &self.random_state) .field("mode", &self.mode) - .field("metrics", &self.metrics) .field("projection", &self.projection) .field("column_indices", &self.column_indices) .field("null_equality", &self.null_equality) @@ -447,24 +487,26 @@ impl HashJoinExec { filter, join_type: *join_type, join_schema, - left_fut: Default::default(), random_state, mode: partition_mode, - metrics: ExecutionPlanMetricsSet::new(), projection, column_indices, null_equality, cache, dynamic_filter: None, + #[cfg(not(feature = "stateless_plan"))] + state: HashJoinExecState::default(), + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), }) } - fn create_dynamic_filter(on: &JoinOn) -> Arc { + fn create_dynamic_filter(on: &JoinOn) -> Arc { // Extract the right-side keys (probe side keys) from the `on` clauses // Dynamic filter will be created from build side values (left side) and applied to probe side (right side) let right_keys: Vec<_> = on.iter().map(|(_, r)| Arc::clone(r)).collect(); // Initialize with a placeholder expression (true) that will be updated when the hash table is built - Arc::new(DynamicFilterPhysicalExpr::new(right_keys, lit(true))) + Arc::new(make_planned_dynamic_filter(lit(true), right_keys)) } /// left (build) side which gets hashed @@ -852,10 +894,8 @@ impl ExecutionPlan for HashJoinExec { filter: self.filter.clone(), join_type: self.join_type, join_schema: Arc::clone(&self.join_schema), - left_fut: Arc::clone(&self.left_fut), random_state: self.random_state.clone(), mode: self.mode, - metrics: ExecutionPlanMetricsSet::new(), projection: self.projection.clone(), column_indices: self.column_indices.clone(), null_equality: self.null_equality, @@ -868,11 +908,15 @@ impl ExecutionPlan for HashJoinExec { self.mode, self.projection.as_ref(), )?, - // Keep the dynamic filter, bounds accumulator will be reset dynamic_filter: self.dynamic_filter.clone(), + #[cfg(not(feature = "stateless_plan"))] + metrics: self.metrics.clone(), + #[cfg(not(feature = "stateless_plan"))] + state: self.state.clone(), })) } + #[cfg(not(feature = "stateless_plan"))] fn reset_state(self: Arc) -> Result> { Ok(Arc::new(HashJoinExec { left: Arc::clone(&self.left), @@ -881,8 +925,6 @@ impl ExecutionPlan for HashJoinExec { filter: self.filter.clone(), join_type: self.join_type, join_schema: Arc::clone(&self.join_schema), - // Reset the left_fut to allow re-execution - left_fut: Arc::new(OnceAsync::default()), random_state: self.random_state.clone(), mode: self.mode, metrics: ExecutionPlanMetricsSet::new(), @@ -891,6 +933,7 @@ impl ExecutionPlan for HashJoinExec { null_equality: self.null_equality, cache: self.cache.clone(), // Reset dynamic filter and bounds accumulator to initial state + state: HashJoinExecState::default(), dynamic_filter: None, })) } @@ -899,7 +942,13 @@ impl ExecutionPlan for HashJoinExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + let on_left = self .on .iter() @@ -921,12 +970,27 @@ impl ExecutionPlan for HashJoinExec { consider using CoalescePartitionsExec or the EnforceDistribution rule" ); + #[cfg(not(feature = "stateless_plan"))] + let exec_state = &self.state; + + #[cfg(feature = "stateless_plan")] + let exec_state = state.get_or_init_state(|| { + HashJoinExecState::new( + // Make executable filter from stored planned filter. + self.dynamic_filter + .as_ref() + .map(|f| Arc::new(f.to_executable())), + ) + }); + let enable_dynamic_filter_pushdown = self.dynamic_filter.is_some(); - let join_metrics = BuildProbeJoinMetrics::new(partition, &self.metrics); + let join_metrics = + BuildProbeJoinMetrics::new(partition, plan_metrics!(self, state)); let left_fut = match self.mode { - PartitionMode::CollectLeft => self.left_fut.try_once(|| { - let left_stream = self.left.execute(0, Arc::clone(&context))?; + PartitionMode::CollectLeft => exec_state.left_fut.try_once(|| { + let left_stream = + execute_input!(0, self.left, 0, Arc::clone(&context), state)?; let reservation = MemoryConsumer::new("HashJoinInput").register(context.memory_pool()); @@ -953,7 +1017,8 @@ impl ExecutionPlan for HashJoinExec { )) })?, PartitionMode::Partitioned => { - let left_stream = self.left.execute(partition, Arc::clone(&context))?; + let left_stream = + execute_input!(0, self.left, partition, Arc::clone(&context), state)?; let reservation = MemoryConsumer::new(format!("HashJoinInput[{partition}]")) @@ -990,19 +1055,25 @@ impl ExecutionPlan for HashJoinExec { let batch_size = context.session_config().batch_size(); + #[cfg(feature = "stateless_plan")] + let df = &exec_state.dynamic_filter; + + #[cfg(not(feature = "stateless_plan"))] + let df = &self.dynamic_filter; + // Initialize build_accumulator lazily with runtime partition counts (only if enabled) // Use RepartitionExec's random state (seeds: 0,0,0,0) for partition routing let repartition_random_state = REPARTITION_RANDOM_STATE; let build_accumulator = enable_dynamic_filter_pushdown .then(|| { - self.dynamic_filter.as_ref().map(|df| { - let filter = Arc::clone(&df.filter); + df.as_ref().map(|df| { + let filter = Arc::clone(df); let on_right = self .on .iter() .map(|(_, right_expr)| Arc::clone(right_expr)) .collect::>(); - Some(Arc::clone(df.build_accumulator.get_or_init(|| { + Some(Arc::clone(exec_state.build_accumulator.get_or_init(|| { Arc::new(SharedBuildAccumulator::new_from_partition_mode( self.mode, self.left.as_ref(), @@ -1019,7 +1090,7 @@ impl ExecutionPlan for HashJoinExec { // we have the batches and the hash map with their keys. We can how create a stream // over the right that uses this information to issue new batches. - let right_stream = self.right.execute(partition, context)?; + let right_stream = execute_input!(1, self.right, partition, context, state)?; // update column indices to reflect the projection let column_indices_after_projection = match &self.projection { @@ -1057,6 +1128,7 @@ impl ExecutionPlan for HashJoinExec { ))) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -1194,7 +1266,7 @@ impl ExecutionPlan for HashJoinExec { // "yes, I can fully evaluate this filter" things might still use it for statistics -> it's worth updating let predicate = Arc::clone(&filter.predicate); if let Ok(dynamic_filter) = - Arc::downcast::(predicate) + Arc::downcast::(predicate) { // We successfully pushed down our self filter - we need to make a new node with the dynamic filter let new_node = Arc::new(HashJoinExec { @@ -1204,18 +1276,17 @@ impl ExecutionPlan for HashJoinExec { filter: self.filter.clone(), join_type: self.join_type, join_schema: Arc::clone(&self.join_schema), - left_fut: Arc::clone(&self.left_fut), random_state: self.random_state.clone(), mode: self.mode, - metrics: ExecutionPlanMetricsSet::new(), projection: self.projection.clone(), column_indices: self.column_indices.clone(), null_equality: self.null_equality, cache: self.cache.clone(), - dynamic_filter: Some(HashJoinExecDynamicFilter { - filter: dynamic_filter, - build_accumulator: OnceLock::new(), - }), + dynamic_filter: Some(dynamic_filter), + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), + #[cfg(not(feature = "stateless_plan"))] + state: self.state.clone(), }); result = result.with_updated_node(new_node as Arc); } @@ -1543,8 +1614,11 @@ async fn collect_left_input( mod tests { use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; + #[cfg(feature = "stateless_plan")] + use crate::execution_plan::execute_plan_preserving_state; + use crate::execution_plan::{execute_plan, execute_plan_and_get_metrics_of}; use crate::joins::hash_join::stream::lookup_join_hashmap; - use crate::test::{TestMemoryExec, assert_join_metrics}; + use crate::test::{TestMemoryExec, assert_join_metrics, collect_batches}; use crate::{ common, expressions::Column, repartition::RepartitionExec, test::build_table_i32, test::exec::MockExec, @@ -1561,10 +1635,13 @@ mod tests { exec_err, internal_err, }; use datafusion_execution::config::SessionConfig; + use datafusion_execution::metrics::MetricsSet; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_expr::Operator; use datafusion_physical_expr::PhysicalExpr; - use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; + use datafusion_physical_expr::expressions::{ + BinaryExpr, DynamicFilterPhysicalExpr, Literal, + }; use hashbrown::HashTable; use insta::{allow_duplicates, assert_snapshot}; use rstest::*; @@ -1640,12 +1717,14 @@ mod tests { null_equality: NullEquality, context: Arc, ) -> Result<(Vec, Vec, MetricsSet)> { - let join = join(left, right, on, join_type, null_equality)?; + let join: Arc = + Arc::new(join(left, right, on, join_type, null_equality)?); let columns_header = columns(&join.schema()); - let stream = join.execute(0, context)?; + let (stream, metrics) = + execute_plan_and_get_metrics_of(Arc::clone(&join), &join, 0, context)?; let batches = common::collect(stream).await?; - let metrics = join.metrics().unwrap(); + let metrics = metrics.unwrap(); Ok((columns_header, batches, metrics)) } @@ -1731,19 +1810,7 @@ mod tests { let columns = columns(&join.schema()); - let mut batches = vec![]; - for i in 0..partition_count { - let stream = join.execute(i, Arc::clone(&context))?; - let more_batches = common::collect(stream).await?; - batches.extend( - more_batches - .into_iter() - .filter(|b| b.num_rows() > 0) - .collect::>(), - ); - } - let metrics = join.metrics().unwrap(); - + let (batches, metrics) = collect_batches(Arc::new(join), context).await?; Ok((columns, batches, metrics)) } @@ -2191,19 +2258,19 @@ mod tests { Arc::new(Column::new_with_schema("b1", &right.schema())?) as _, )]; - let join = join( + let join: Arc = Arc::new(join( left, right, on, &JoinType::Inner, NullEquality::NullEqualsNothing, - )?; + )?); let columns = columns(&join.schema()); assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]); // first part - let stream = join.execute(0, Arc::clone(&task_ctx))?; + let stream = execute_plan(Arc::clone(&join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; let expected_batch_count = if cfg!(not(feature = "force_hash_collisions")) { @@ -2238,7 +2305,7 @@ mod tests { } // second part - let stream = join.execute(1, Arc::clone(&task_ctx))?; + let stream = execute_plan(Arc::clone(&join), 1, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; let expected_batch_count = if cfg!(not(feature = "force_hash_collisions")) { @@ -2312,7 +2379,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]); - let stream = join.execute(0, task_ctx).unwrap(); + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx)).unwrap(); let batches = common::collect(stream).await.unwrap(); allow_duplicates! { @@ -2362,7 +2429,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]); - let stream = join.execute(0, task_ctx).unwrap(); + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx)).unwrap(); let batches = common::collect(stream).await.unwrap(); allow_duplicates! { @@ -2410,7 +2477,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]); - let stream = join.execute(0, task_ctx).unwrap(); + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx)).unwrap(); let batches = common::collect(stream).await.unwrap(); allow_duplicates! { @@ -2454,7 +2521,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]); - let stream = join.execute(0, task_ctx).unwrap(); + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx)).unwrap(); let batches = common::collect(stream).await.unwrap(); allow_duplicates! { @@ -2609,7 +2676,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a1", "b1", "c1"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; // ignore the order @@ -2672,7 +2739,7 @@ mod tests { let columns_header = columns(&join.schema()); assert_eq!(columns_header.clone(), vec!["a1", "b1", "c1"]); - let stream = join.execute(0, Arc::clone(&task_ctx))?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; allow_duplicates! { @@ -2711,7 +2778,7 @@ mod tests { let columns_header = columns(&join.schema()); assert_eq!(columns_header, vec!["a1", "b1", "c1"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; allow_duplicates! { @@ -2751,7 +2818,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a2", "b2", "c2"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; // RightSemi join output is expected to preserve right input order @@ -2814,7 +2881,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a2", "b2", "c2"]); - let stream = join.execute(0, Arc::clone(&task_ctx))?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; // RightSemi join output is expected to preserve right input order @@ -2851,7 +2918,8 @@ mod tests { &JoinType::RightSemi, NullEquality::NullEqualsNothing, )?; - let stream = join.execute(0, task_ctx)?; + + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; // RightSemi join output is expected to preserve right input order @@ -2892,7 +2960,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a1", "b1", "c1"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; allow_duplicates! { @@ -2952,7 +3020,7 @@ mod tests { let columns_header = columns(&join.schema()); assert_eq!(columns_header, vec!["a1", "b1", "c1"]); - let stream = join.execute(0, Arc::clone(&task_ctx))?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; allow_duplicates! { @@ -2995,7 +3063,7 @@ mod tests { let columns_header = columns(&join.schema()); assert_eq!(columns_header, vec!["a1", "b1", "c1"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; allow_duplicates! { @@ -3038,7 +3106,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a2", "b2", "c2"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; // RightAnti join output is expected to preserve right input order @@ -3099,7 +3167,7 @@ mod tests { let columns_header = columns(&join.schema()); assert_eq!(columns_header, vec!["a2", "b2", "c2"]); - let stream = join.execute(0, Arc::clone(&task_ctx))?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; // RightAnti join output is expected to preserve right input order @@ -3146,7 +3214,7 @@ mod tests { let columns_header = columns(&join.schema()); assert_eq!(columns_header, vec!["a2", "b2", "c2"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; // RightAnti join output is expected to preserve right input order @@ -3292,7 +3360,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; allow_duplicates! { @@ -3659,7 +3727,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a", "b", "c", "a", "b", "c"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; allow_duplicates! { @@ -3736,7 +3804,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a", "b", "c", "a", "b", "c"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; allow_duplicates! { @@ -3785,7 +3853,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a", "b", "c", "a", "b", "c"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; allow_duplicates! { @@ -3837,7 +3905,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a", "b", "c", "a", "b", "c"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; allow_duplicates! { @@ -3888,7 +3956,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a", "b", "c", "a", "b", "c"]); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; let expected = [ @@ -4092,7 +4160,7 @@ mod tests { )?; let task_ctx = Arc::new(TaskContext::default()); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; allow_duplicates! { @@ -4153,7 +4221,7 @@ mod tests { .unwrap(); let task_ctx = Arc::new(TaskContext::default()); - let stream = join.execute(0, task_ctx).unwrap(); + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx)).unwrap(); // Expect that an error is returned let result_string = common::collect(stream).await.unwrap_err().to_string(); @@ -4266,7 +4334,8 @@ mod tests { ) .unwrap(); - let stream = join.execute(0, task_ctx).unwrap(); + let stream = + execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx)).unwrap(); let batches = common::collect(stream).await.unwrap(); // For inner/right join expected batch count equals dev_ceil result, @@ -4356,7 +4425,7 @@ mod tests { NullEquality::NullEqualsNothing, )?; - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, Arc::clone(&task_ctx))?; let err = common::collect(stream).await.unwrap_err(); // Asserting that operator-level reservation attempting to overallocate @@ -4437,7 +4506,7 @@ mod tests { NullEquality::NullEqualsNothing, )?; - let stream = join.execute(1, task_ctx)?; + let stream = execute_plan(Arc::new(join), 1, Arc::clone(&task_ctx))?; let err = common::collect(stream).await.unwrap_err(); // Asserting that stream-level reservation attempting to overallocate @@ -4586,6 +4655,40 @@ mod tests { schema.fields().iter().map(|f| f.name().clone()).collect() } + #[cfg(not(feature = "stateless_plan"))] + async fn collect_and_get_dynamic_filter( + join: HashJoinExec, + task_ctx: Arc, + ) -> Result> { + let join = Arc::new(join); + let _batches = crate::collect(Arc::clone(&join) as Arc<_>, task_ctx).await?; + Ok(join.dynamic_filter.clone().unwrap()) + } + + #[cfg(feature = "stateless_plan")] + async fn collect_and_get_dynamic_filter( + join: HashJoinExec, + task_ctx: Arc, + ) -> Result> { + let dynamic_filter = join.dynamic_filter.clone().unwrap(); + // Execute the join + let (stream, state) = + execute_plan_preserving_state(Arc::new(join), 0, Arc::clone(&task_ctx)) + .as_result() + .unwrap() + .into_parts(); + let _batches = common::collect(stream).await?; + Ok(Arc::new( + state + .planned_dynamic_filter_to_executable(dynamic_filter) + .unwrap() + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + )) + } + /// This test verifies that the dynamic filter is marked as complete after HashJoinExec finishes building the hash table. #[tokio::test] async fn test_hash_join_marks_filter_complete() -> Result<()> { @@ -4608,7 +4711,6 @@ mod tests { // Create a dynamic filter manually let dynamic_filter = HashJoinExec::create_dynamic_filter(&on); - let dynamic_filter_clone = Arc::clone(&dynamic_filter); // Create HashJoinExec with the dynamic filter let mut join = HashJoinExec::try_new( @@ -4621,18 +4723,12 @@ mod tests { PartitionMode::CollectLeft, NullEquality::NullEqualsNothing, )?; - join.dynamic_filter = Some(HashJoinExecDynamicFilter { - filter: dynamic_filter, - build_accumulator: OnceLock::new(), - }); - - // Execute the join - let stream = join.execute(0, task_ctx)?; - let _batches = common::collect(stream).await?; + join.dynamic_filter = Some(Arc::clone(&dynamic_filter)); + let filter = collect_and_get_dynamic_filter(join, task_ctx).await?; // After the join completes, the dynamic filter should be marked as complete // wait_complete() should return immediately - dynamic_filter_clone.wait_complete().await; + filter.wait_complete().await; Ok(()) } @@ -4656,7 +4752,6 @@ mod tests { // Create a dynamic filter manually let dynamic_filter = HashJoinExec::create_dynamic_filter(&on); - let dynamic_filter_clone = Arc::clone(&dynamic_filter); // Create HashJoinExec with the dynamic filter let mut join = HashJoinExec::try_new( @@ -4669,18 +4764,14 @@ mod tests { PartitionMode::CollectLeft, NullEquality::NullEqualsNothing, )?; - join.dynamic_filter = Some(HashJoinExecDynamicFilter { - filter: dynamic_filter, - build_accumulator: OnceLock::new(), - }); - // Execute the join - let stream = join.execute(0, task_ctx)?; - let _batches = common::collect(stream).await?; + join.dynamic_filter = Some(Arc::clone(&dynamic_filter)); + // Execute the join + let filter = collect_and_get_dynamic_filter(join, task_ctx).await?; // Even with empty build side, the dynamic filter should be marked as complete // wait_complete() should return immediately - dynamic_filter_clone.wait_complete().await; + filter.wait_complete().await; Ok(()) } diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 76dca7239114b..d37d4adbc25f4 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -37,12 +37,14 @@ use crate::joins::utils::{ need_produce_right_in_final, }; use crate::metrics::{ - Count, ExecutionPlanMetricsSet, MetricBuilder, MetricType, MetricsSet, RatioMetrics, + Count, ExecutionPlanMetricsSet, MetricBuilder, MetricType, RatioMetrics, }; use crate::projection::{ EmbeddedProjection, JoinData, ProjectionExec, try_embed_projection, try_pushdown_through_join, }; +#[cfg(feature = "stateless_plan")] +use crate::state::{PlanState, PlanStateNode}; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, @@ -66,6 +68,8 @@ use datafusion_common::{ }; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::MetricsSet; use datafusion_expr::JoinType; use datafusion_physical_expr::equivalence::{ ProjectionMapping, join_equivalence_properties, @@ -182,6 +186,22 @@ pub struct NestedLoopJoinExec { /// The full concatenated schema of left and right children should be distinct from /// the output schema of the operator join_schema: SchemaRef, + /// Information of index and left / right placement of columns + column_indices: Vec, + /// Projection to apply to the output of the join + projection: Option>, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, + /// State of the plan. + #[cfg(not(feature = "stateless_plan"))] + state: NestedLoopJoinExecState, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, +} + +#[derive(Debug, Default)] +struct NestedLoopJoinExecState { /// Future that consumes left input and buffers it in memory /// /// This structure is *shared* across all output streams. @@ -189,15 +209,13 @@ pub struct NestedLoopJoinExec { /// Each output stream waits on the `OnceAsync` to signal the completion of /// the build(left) side data, and buffer them all for later joining. build_side_data: OnceAsync, - /// Information of index and left / right placement of columns - column_indices: Vec, - /// Projection to apply to the output of the join - projection: Option>, +} - /// Execution metrics - metrics: ExecutionPlanMetricsSet, - /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanProperties, +#[cfg(feature = "stateless_plan")] +impl PlanState for NestedLoopJoinExecState { + fn as_any(&self) -> &dyn Any { + self + } } impl NestedLoopJoinExec { @@ -229,11 +247,13 @@ impl NestedLoopJoinExec { filter, join_type: *join_type, join_schema, - build_side_data: Default::default(), column_indices, projection, - metrics: Default::default(), cache, + #[cfg(not(feature = "stateless_plan"))] + state: Default::default(), + #[cfg(not(feature = "stateless_plan"))] + metrics: Default::default(), }) } @@ -489,7 +509,13 @@ impl ExecutionPlan for NestedLoopJoinExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + assert_eq_or_internal_err!( self.left.output_partitioning().partition_count(), 1, @@ -497,15 +523,23 @@ impl ExecutionPlan for NestedLoopJoinExec { consider using CoalescePartitionsExec or the EnforceDistribution rule" ); - let metrics = NestedLoopJoinMetrics::new(&self.metrics, partition); + let metrics = NestedLoopJoinMetrics::new(plan_metrics!(self, state), partition); // Initialization reservation for load of inner table let load_reservation = MemoryConsumer::new(format!("NestedLoopJoinLoad[{partition}]")) .register(context.memory_pool()); - let build_side_data = self.build_side_data.try_once(|| { - let stream = self.left.execute(0, Arc::clone(&context))?; + #[cfg(feature = "stateless_plan")] + let build_side_data = &state + .get_or_init_state(NestedLoopJoinExecState::default) + .build_side_data; + + #[cfg(not(feature = "stateless_plan"))] + let build_side_data = &self.state.build_side_data; + + let build_side_data = build_side_data.try_once(|| { + let stream = execute_input!(0, self.left, 0, Arc::clone(&context), state)?; Ok(collect_left_input( stream, @@ -518,7 +552,7 @@ impl ExecutionPlan for NestedLoopJoinExec { let batch_size = context.session_config().batch_size(); - let probe_side_data = self.right.execute(partition, context)?; + let probe_side_data = execute_input!(1, self.right, partition, context, state)?; // update column indices to reflect the projection let column_indices_after_projection = match &self.projection { @@ -541,6 +575,7 @@ impl ExecutionPlan for NestedLoopJoinExec { ))) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -2213,15 +2248,16 @@ fn build_unmatched_batch( #[cfg(test)] pub(crate) mod tests { use super::*; - use crate::test::{TestMemoryExec, assert_join_metrics}; + use crate::test::{TestMemoryExec, assert_join_metrics, collect_with}; use crate::{ - common, expressions::Column, repartition::RepartitionExec, test::build_table_i32, + expressions::Column, repartition::RepartitionExec, test::build_table_i32, }; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field}; use datafusion_common::test_util::batches_to_sort_string; use datafusion_common::{ScalarValue, assert_contains}; + use datafusion_execution::metrics::MetricsSet; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; @@ -2361,21 +2397,15 @@ pub(crate) mod tests { NestedLoopJoinExec::try_new(left, right, join_filter, join_type, None)?; let columns = columns(&nested_loop_join.schema()); let mut batches = vec![]; - for i in 0..partition_count { - let stream = nested_loop_join.execute(i, Arc::clone(&context))?; - let more_batches = common::collect(stream).await?; - batches.extend( - more_batches - .into_iter() - .inspect(|b| { - assert!(b.num_rows() <= context.session_config().batch_size()) - }) - .filter(|b| b.num_rows() > 0) - .collect::>(), - ); - } + let batch_size = context.session_config().batch_size(); - let metrics = nested_loop_join.metrics().unwrap(); + let metrics = collect_with(Arc::new(nested_loop_join), context, |_, batch| { + let batch = batch?; + assert!(batch.num_rows() <= batch_size); + batches.push(batch); + Ok(()) + }) + .await?; Ok((columns, batches, metrics)) } diff --git a/datafusion/physical-plan/src/joins/piecewise_merge_join/classic_join.rs b/datafusion/physical-plan/src/joins/piecewise_merge_join/classic_join.rs index 04daa3698d920..c866fb955adb6 100644 --- a/datafusion/physical-plan/src/joins/piecewise_merge_join/classic_join.rs +++ b/datafusion/physical-plan/src/joins/piecewise_merge_join/classic_join.rs @@ -655,6 +655,7 @@ mod tests { use super::*; use crate::{ ExecutionPlan, common, + execution_plan::execute_plan, joins::PiecewiseMergeJoinExec, test::{TestMemoryExec, build_table_i32}, }; @@ -762,7 +763,7 @@ mod tests { let join = join(left, right, on, operator, join_type)?; let columns = columns(&join.schema()); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, task_ctx)?; let batches = common::collect(stream).await?; Ok((columns, batches)) } diff --git a/datafusion/physical-plan/src/joins/piecewise_merge_join/exec.rs b/datafusion/physical-plan/src/joins/piecewise_merge_join/exec.rs index 508be2e3984f4..75d850a8dbd92 100644 --- a/datafusion/physical-plan/src/joins/piecewise_merge_join/exec.rs +++ b/datafusion/physical-plan/src/joins/piecewise_merge_join/exec.rs @@ -24,6 +24,8 @@ use arrow::{ use arrow_schema::{SchemaRef, SortOptions}; use datafusion_common::not_impl_err; use datafusion_common::{JoinSide, Result, internal_err}; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_execution::{ SendableRecordBatchStream, memory_pool::{MemoryConsumer, MemoryReservation}, @@ -50,7 +52,8 @@ use crate::joins::piecewise_merge_join::utils::{ build_visited_indices_map, is_existence_join, is_right_existence_join, }; use crate::joins::utils::asymmetric_join_output_partitioning; -use crate::metrics::MetricsSet; +#[cfg(feature = "stateless_plan")] +use crate::state::{PlanState, PlanStateNode}; use crate::{DisplayAs, DisplayFormatType, ExecutionPlanProperties}; use crate::{ ExecutionPlan, PlanProperties, @@ -58,7 +61,6 @@ use crate::{ SharedBitmapBuilder, utils::{BuildProbeJoinMetrics, OnceAsync, OnceFut, build_join_schema}, }, - metrics::ExecutionPlanMetricsSet, spill::get_record_batch_memory_size, }; @@ -86,7 +88,7 @@ use crate::{ /// Both sides are sorted so that we can iterate from index 0 to the end on each side. This ordering ensures /// that when we find the first matching pair of rows, we can emit the current stream row joined with all remaining /// probe rows from the match position onward, without rescanning earlier probe rows. -/// +/// /// For `<` and `<=` operators, both inputs are sorted in **descending** order, while for `>` and `>=` operators /// they are sorted in **ascending** order. This choice ensures that the pointer on the buffered side can advance /// monotonically as we stream new batches from the stream side. @@ -129,34 +131,34 @@ use crate::{ /// /// Processing Row 1: /// -/// Sorted Buffered Side Sorted Streamed Side -/// ┌──────────────────┐ ┌──────────────────┐ -/// 1 │ 100 │ 1 │ 100 │ -/// ├──────────────────┤ ├──────────────────┤ -/// 2 │ 200 │ ─┐ 2 │ 200 │ -/// ├──────────────────┤ │ For row 1 on streamed side with ├──────────────────┤ -/// 3 │ 200 │ │ value 100, we emit rows 2 - 5. 3 │ 500 │ +/// Sorted Buffered Side Sorted Streamed Side +/// ┌──────────────────┐ ┌──────────────────┐ +/// 1 │ 100 │ 1 │ 100 │ +/// ├──────────────────┤ ├──────────────────┤ +/// 2 │ 200 │ ─┐ 2 │ 200 │ +/// ├──────────────────┤ │ For row 1 on streamed side with ├──────────────────┤ +/// 3 │ 200 │ │ value 100, we emit rows 2 - 5. 3 │ 500 │ /// ├──────────────────┤ │ as matches when the operator is └──────────────────┘ /// 4 │ 300 │ │ `Operator::Lt` (<) Emitting all /// ├──────────────────┤ │ rows after the first match (row /// 5 │ 400 │ ─┘ 2 buffered side; 100 < 200) -/// └──────────────────┘ +/// └──────────────────┘ /// /// Processing Row 2: /// By sorting the streamed side we know /// -/// Sorted Buffered Side Sorted Streamed Side -/// ┌──────────────────┐ ┌──────────────────┐ -/// 1 │ 100 │ 1 │ 100 │ -/// ├──────────────────┤ ├──────────────────┤ -/// 2 │ 200 │ <- Start here when probing for the 2 │ 200 │ -/// ├──────────────────┤ streamed side row 2. ├──────────────────┤ -/// 3 │ 200 │ 3 │ 500 │ +/// Sorted Buffered Side Sorted Streamed Side +/// ┌──────────────────┐ ┌──────────────────┐ +/// 1 │ 100 │ 1 │ 100 │ +/// ├──────────────────┤ ├──────────────────┤ +/// 2 │ 200 │ <- Start here when probing for the 2 │ 200 │ +/// ├──────────────────┤ streamed side row 2. ├──────────────────┤ +/// 3 │ 200 │ 3 │ 500 │ /// ├──────────────────┤ └──────────────────┘ -/// 4 │ 300 │ -/// ├──────────────────┤ +/// 4 │ 300 │ +/// ├──────────────────┤ /// 5 │ 400 │ -/// └──────────────────┘ +/// └──────────────────┘ /// ``` /// /// ## Existence Joins (Semi, Anti, Mark) @@ -202,10 +204,10 @@ use crate::{ /// 1 │ 100 │ 1 │ 500 │ /// ├──────────────────┤ ├──────────────────┤ /// 2 │ 200 │ 2 │ 200 │ -/// ├──────────────────┤ ├──────────────────┤ +/// ├──────────────────┤ ├──────────────────┤ /// 3 │ 200 │ 3 │ 300 │ /// ├──────────────────┤ └──────────────────┘ -/// 4 │ 300 │ ─┐ +/// 4 │ 300 │ ─┐ /// ├──────────────────┤ | We emit matches for row 4 - 5 /// 5 │ 400 │ ─┘ on the buffered side. /// └──────────────────┘ @@ -236,11 +238,11 @@ use crate::{ /// /// # Mark Join: /// Sorts the probe side, then computes the min/max range of the probe keys and scans the buffered side only -/// within that range. +/// within that range. /// Complexity: `O(|S| + scan(R[range]))`. /// /// ## Nested Loop Join -/// Compares every row from `S` with every row from `R`. +/// Compares every row from `S` with every row from `R`. /// Complexity: `O(|S| * |R|)`. /// /// ## Nested Loop Join @@ -262,10 +264,6 @@ pub struct PiecewiseMergeJoinExec { pub join_type: JoinType, /// The schema once the join is applied schema: SchemaRef, - /// Buffered data - buffered_fut: OnceAsync, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// Sort expressions - See above for more details [`PiecewiseMergeJoinExec`] /// @@ -282,6 +280,26 @@ pub struct PiecewiseMergeJoinExec { cache: PlanProperties, /// Number of partitions to process num_partitions: usize, + + /// Plan state + #[cfg(not(feature = "stateless_plan"))] + state: PiecewiseMergeJoinExecState, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, +} + +#[derive(Debug, Default)] +struct PiecewiseMergeJoinExecState { + /// Buffered data + buffered_fut: OnceAsync, +} + +#[cfg(feature = "stateless_plan")] +impl PlanState for PiecewiseMergeJoinExecState { + fn as_any(&self) -> &dyn std::any::Any { + self + } } impl PiecewiseMergeJoinExec { @@ -368,13 +386,17 @@ impl PiecewiseMergeJoinExec { operator, join_type, schema, - buffered_fut: Default::default(), - metrics: ExecutionPlanMetricsSet::new(), + left_child_plan_required_order, right_batch_required_orders, sort_options, cache, num_partitions, + + #[cfg(not(feature = "stateless_plan"))] + state: Default::default(), + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), }) } @@ -531,16 +553,31 @@ impl ExecutionPlan for PiecewiseMergeJoinExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + let on_buffered = Arc::clone(&self.on.0); let on_streamed = Arc::clone(&self.on.1); - let metrics = BuildProbeJoinMetrics::new(partition, &self.metrics); - let buffered_fut = self.buffered_fut.try_once(|| { + #[cfg(feature = "stateless_plan")] + let buffered_fut = &state + .get_or_init_state(PiecewiseMergeJoinExecState::default) + .buffered_fut; + + #[cfg(not(feature = "stateless_plan"))] + let buffered_fut = &self.state.buffered_fut; + + let metrics = BuildProbeJoinMetrics::new(partition, plan_metrics!(self, state)); + let buffered_fut = buffered_fut.try_once(|| { let reservation = MemoryConsumer::new("PiecewiseMergeJoinInput") .register(context.memory_pool()); - let buffered_stream = self.buffered.execute(0, Arc::clone(&context))?; + let buffered_stream = + execute_input!(0, self.buffered, 0, Arc::clone(&context), state)?; Ok(build_buffered_data( buffered_stream, Arc::clone(&on_buffered), @@ -551,7 +588,8 @@ impl ExecutionPlan for PiecewiseMergeJoinExec { )) })?; - let streamed = self.streamed.execute(partition, Arc::clone(&context))?; + let streamed = + execute_input!(1, self.streamed, partition, Arc::clone(&context), state)?; let batch_size = context.session_config().batch_size(); @@ -574,6 +612,7 @@ impl ExecutionPlan for PiecewiseMergeJoinExec { } } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/exec.rs b/datafusion/physical-plan/src/joins/sort_merge_join/exec.rs index 5362259d22ea8..d8d7b6098c5df 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/exec.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/exec.rs @@ -32,11 +32,12 @@ use crate::joins::utils::{ estimate_join_statistics, reorder_output_after_swap, symmetric_join_output_partitioning, }; -use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::projection::{ ProjectionExec, join_allows_pushdown, join_table_borders, new_join_children, physical_to_column_exprs, update_join_on, }; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, PlanProperties, SendableRecordBatchStream, Statistics, @@ -50,6 +51,8 @@ use datafusion_common::{ }; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::MemoryConsumer; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr_common::physical_expr::{PhysicalExprRef, fmt_sql}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, OrderingRequirements}; @@ -116,8 +119,6 @@ pub struct SortMergeJoinExec { pub join_type: JoinType, /// The schema once the join is applied schema: SchemaRef, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// The left SortExpr left_sort_exprs: LexOrdering, /// The right SortExpr @@ -128,6 +129,9 @@ pub struct SortMergeJoinExec { pub null_equality: NullEquality, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } impl SortMergeJoinExec { @@ -193,12 +197,13 @@ impl SortMergeJoinExec { filter, join_type, schema, - metrics: ExecutionPlanMetricsSet::new(), left_sort_exprs, right_sort_exprs, sort_options, null_equality, cache, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), }) } @@ -457,7 +462,13 @@ impl ExecutionPlan for SortMergeJoinExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + let left_partitions = self.left.output_partitioning().partition_count(); let right_partitions = self.right.output_partitioning().partition_count(); assert_eq_or_internal_err!( @@ -467,13 +478,14 @@ impl ExecutionPlan for SortMergeJoinExec { consider using RepartitionExec" ); let (on_left, on_right) = self.on.iter().cloned().unzip(); - let (streamed, buffered, on_streamed, on_buffered) = + let (streamed, buffered, on_streamed, on_buffered, streamed_child_num) = if SortMergeJoinExec::probe_side(&self.join_type) == JoinSide::Left { ( Arc::clone(&self.left), Arc::clone(&self.right), on_left, on_right, + 0, ) } else { ( @@ -481,12 +493,27 @@ impl ExecutionPlan for SortMergeJoinExec { Arc::clone(&self.left), on_right, on_left, + 1, ) }; // execute children plans - let streamed = streamed.execute(partition, Arc::clone(&context))?; - let buffered = buffered.execute(partition, Arc::clone(&context))?; + let streamed = execute_input!( + streamed_child_num, + streamed, + partition, + Arc::clone(&context), + state + )?; + let buffered = execute_input!( + 1 - streamed_child_num, + buffered, + partition, + Arc::clone(&context), + state + )?; + + let _ = streamed_child_num; // create output buffer let batch_size = context.session_config().batch_size(); @@ -508,12 +535,13 @@ impl ExecutionPlan for SortMergeJoinExec { self.filter.clone(), self.join_type, batch_size, - SortMergeJoinMetrics::new(partition, &self.metrics), + SortMergeJoinMetrics::new(partition, plan_metrics!(self, state)), reservation, context.runtime_env(), )?)) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs index 171b6e5d682ad..669224a2a777a 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs @@ -51,6 +51,7 @@ use datafusion_physical_expr::expressions::BinaryExpr; use insta::{allow_duplicates, assert_snapshot}; use crate::{ + execution_plan::{execute_plan, execute_plan_and_get_metrics_of}, expressions::Column, joins::sort_merge_join::stream::{JoinedRecordBatches, get_corrected_filter_mask}, }; @@ -304,7 +305,7 @@ async fn join_collect_with_filter( )?; let columns = columns(&join.schema()); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, task_ctx)?; let batches = common::collect(stream).await?; Ok((columns, batches)) } @@ -322,7 +323,7 @@ async fn join_collect_with_options( join_with_options(left, right, on, join_type, sort_options, null_equality)?; let columns = columns(&join.schema()); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, task_ctx)?; let batches = common::collect(stream).await?; Ok((columns, batches)) } @@ -339,7 +340,7 @@ async fn join_collect_batch_size_equals_two( let join = join(left, right, on, join_type)?; let columns = columns(&join.schema()); - let stream = join.execute(0, task_ctx)?; + let stream = execute_plan(Arc::new(join), 0, task_ctx)?; let batches = common::collect(stream).await?; Ok((columns, batches)) } @@ -2075,25 +2076,27 @@ async fn overallocation_single_batch_no_spill() -> Result<()> { .with_runtime(Arc::clone(&runtime)); let task_ctx = Arc::new(task_ctx); - let join = join_with_options( + let join: Arc = Arc::new(join_with_options( Arc::clone(&left), Arc::clone(&right), on.clone(), join_type, sort_options.clone(), NullEquality::NullEqualsNothing, - )?; + )?); - let stream = join.execute(0, task_ctx)?; + let (stream, metrics) = + execute_plan_and_get_metrics_of(Arc::clone(&join), &join, 0, task_ctx)?; let err = common::collect(stream).await.unwrap_err(); assert_contains!(err.to_string(), "Failed to allocate additional"); assert_contains!(err.to_string(), "SMJStream[0]"); assert_contains!(err.to_string(), "Disk spilling disabled"); - assert!(join.metrics().is_some()); - assert_eq!(join.metrics().unwrap().spill_count(), Some(0)); - assert_eq!(join.metrics().unwrap().spilled_bytes(), Some(0)); - assert_eq!(join.metrics().unwrap().spilled_rows(), Some(0)); + + let metrics = metrics.unwrap(); + assert_eq!(metrics.spill_count(), Some(0)); + assert_eq!(metrics.spilled_bytes(), Some(0)); + assert_eq!(metrics.spilled_rows(), Some(0)); } Ok(()) @@ -2155,25 +2158,27 @@ async fn overallocation_multi_batch_no_spill() -> Result<()> { .with_session_config(session_config.clone()) .with_runtime(Arc::clone(&runtime)); let task_ctx = Arc::new(task_ctx); - let join = join_with_options( + let join: Arc = Arc::new(join_with_options( Arc::clone(&left), Arc::clone(&right), on.clone(), join_type, sort_options.clone(), NullEquality::NullEqualsNothing, - )?; + )?); - let stream = join.execute(0, task_ctx)?; + let (stream, metrics) = + execute_plan_and_get_metrics_of(Arc::clone(&join), &join, 0, task_ctx)?; let err = common::collect(stream).await.unwrap_err(); assert_contains!(err.to_string(), "Failed to allocate additional"); assert_contains!(err.to_string(), "SMJStream[0]"); assert_contains!(err.to_string(), "Disk spilling disabled"); - assert!(join.metrics().is_some()); - assert_eq!(join.metrics().unwrap().spill_count(), Some(0)); - assert_eq!(join.metrics().unwrap().spilled_bytes(), Some(0)); - assert_eq!(join.metrics().unwrap().spilled_rows(), Some(0)); + + let metrics = metrics.unwrap(); + assert_eq!(metrics.spill_count(), Some(0)); + assert_eq!(metrics.spilled_bytes(), Some(0)); + assert_eq!(metrics.spilled_rows(), Some(0)); } Ok(()) @@ -2218,43 +2223,49 @@ async fn overallocation_single_batch_spill() -> Result<()> { .with_runtime(Arc::clone(&runtime)); let task_ctx = Arc::new(task_ctx); - let join = join_with_options( + let join: Arc = Arc::new(join_with_options( Arc::clone(&left), Arc::clone(&right), on.clone(), *join_type, sort_options.clone(), NullEquality::NullEqualsNothing, - )?; + )?); - let stream = join.execute(0, task_ctx)?; + let (stream, metrics) = + execute_plan_and_get_metrics_of(Arc::clone(&join), &join, 0, task_ctx)?; let spilled_join_result = common::collect(stream).await.unwrap(); - assert!(join.metrics().is_some()); - assert!(join.metrics().unwrap().spill_count().unwrap() > 0); - assert!(join.metrics().unwrap().spilled_bytes().unwrap() > 0); - assert!(join.metrics().unwrap().spilled_rows().unwrap() > 0); + let metrics = metrics.unwrap(); + assert!(metrics.spill_count().unwrap() > 0); + assert!(metrics.spilled_bytes().unwrap() > 0); + assert!(metrics.spilled_rows().unwrap() > 0); // Run the test with no spill configuration as let task_ctx_no_spill = TaskContext::default().with_session_config(session_config.clone()); let task_ctx_no_spill = Arc::new(task_ctx_no_spill); - let join = join_with_options( + let join: Arc = Arc::new(join_with_options( Arc::clone(&left), Arc::clone(&right), on.clone(), *join_type, sort_options.clone(), NullEquality::NullEqualsNothing, + )?); + let (stream, metrics) = execute_plan_and_get_metrics_of( + Arc::clone(&join), + &join, + 0, + task_ctx_no_spill, )?; - let stream = join.execute(0, task_ctx_no_spill)?; let no_spilled_join_result = common::collect(stream).await.unwrap(); - assert!(join.metrics().is_some()); - assert_eq!(join.metrics().unwrap().spill_count(), Some(0)); - assert_eq!(join.metrics().unwrap().spilled_bytes(), Some(0)); - assert_eq!(join.metrics().unwrap().spilled_rows(), Some(0)); + let metrics = metrics.unwrap(); + assert_eq!(metrics.spill_count(), Some(0)); + assert_eq!(metrics.spilled_bytes(), Some(0)); + assert_eq!(metrics.spilled_rows(), Some(0)); // Compare spilled and non spilled data to check spill logic doesn't corrupt the data assert_eq!(spilled_join_result, no_spilled_join_result); } @@ -2321,42 +2332,48 @@ async fn overallocation_multi_batch_spill() -> Result<()> { .with_session_config(session_config.clone()) .with_runtime(Arc::clone(&runtime)); let task_ctx = Arc::new(task_ctx); - let join = join_with_options( + let join: Arc = Arc::new(join_with_options( Arc::clone(&left), Arc::clone(&right), on.clone(), *join_type, sort_options.clone(), NullEquality::NullEqualsNothing, - )?; + )?); - let stream = join.execute(0, task_ctx)?; + let (stream, metrics) = + execute_plan_and_get_metrics_of(Arc::clone(&join), &join, 0, task_ctx)?; let spilled_join_result = common::collect(stream).await.unwrap(); - assert!(join.metrics().is_some()); - assert!(join.metrics().unwrap().spill_count().unwrap() > 0); - assert!(join.metrics().unwrap().spilled_bytes().unwrap() > 0); - assert!(join.metrics().unwrap().spilled_rows().unwrap() > 0); + let metrics = metrics.unwrap(); + assert!(metrics.spill_count().unwrap() > 0); + assert!(metrics.spilled_bytes().unwrap() > 0); + assert!(metrics.spilled_rows().unwrap() > 0); // Run the test with no spill configuration as let task_ctx_no_spill = TaskContext::default().with_session_config(session_config.clone()); let task_ctx_no_spill = Arc::new(task_ctx_no_spill); - let join = join_with_options( + let join: Arc = Arc::new(join_with_options( Arc::clone(&left), Arc::clone(&right), on.clone(), *join_type, sort_options.clone(), NullEquality::NullEqualsNothing, + )?); + let (stream, metrics) = execute_plan_and_get_metrics_of( + Arc::clone(&join), + &join, + 0, + task_ctx_no_spill, )?; - let stream = join.execute(0, task_ctx_no_spill)?; let no_spilled_join_result = common::collect(stream).await.unwrap(); - assert!(join.metrics().is_some()); - assert_eq!(join.metrics().unwrap().spill_count(), Some(0)); - assert_eq!(join.metrics().unwrap().spilled_bytes(), Some(0)); - assert_eq!(join.metrics().unwrap().spilled_rows(), Some(0)); + let metrics = metrics.unwrap(); + assert_eq!(metrics.spill_count(), Some(0)); + assert_eq!(metrics.spilled_bytes(), Some(0)); + assert_eq!(metrics.spilled_rows(), Some(0)); // Compare spilled and non spilled data to check spill logic doesn't corrupt the data assert_eq!(spilled_join_result, no_spilled_join_result); } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 1f6bc703a0300..fa51093ac55a2 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -50,11 +50,12 @@ use crate::projection::{ ProjectionExec, join_allows_pushdown, join_table_borders, new_join_children, physical_to_column_exprs, update_join_filter, update_join_on, }; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, joins::StreamJoinPartitionMode, - metrics::{ExecutionPlanMetricsSet, MetricsSet}, }; use arrow::array::{ @@ -72,6 +73,8 @@ use datafusion_common::{ }; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::MemoryConsumer; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; @@ -184,8 +187,6 @@ pub struct SymmetricHashJoinExec { pub(crate) join_type: JoinType, /// Shares the `RandomState` for the hashing algorithm random_state: RandomState, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// Information of index and left / right placement of columns column_indices: Vec, /// Defines the null equality for the join. @@ -198,6 +199,9 @@ pub struct SymmetricHashJoinExec { mode: StreamJoinPartitionMode, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } impl SymmetricHashJoinExec { @@ -247,13 +251,14 @@ impl SymmetricHashJoinExec { filter, join_type: *join_type, random_state, - metrics: ExecutionPlanMetricsSet::new(), column_indices, null_equality, left_sort_exprs, right_sort_exprs, mode, cache, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), }) } @@ -466,6 +471,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { )?)) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -479,7 +485,13 @@ impl ExecutionPlan for SymmetricHashJoinExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + let left_partitions = self.left.output_partitioning().partition_count(); let right_partitions = self.right.output_partitioning().partition_count(); assert_eq_or_internal_err!( @@ -517,9 +529,10 @@ impl ExecutionPlan for SymmetricHashJoinExec { let right_side_joiner = OneSideHashJoiner::new(JoinSide::Right, on_right, self.right.schema()); - let left_stream = self.left.execute(partition, Arc::clone(&context))?; - - let right_stream = self.right.execute(partition, Arc::clone(&context))?; + let left_stream = + execute_input!(0, self.left, partition, Arc::clone(&context), state)?; + let right_stream = + execute_input!(1, self.right, partition, Arc::clone(&context), state)?; let batch_size = context.session_config().batch_size(); let enforce_batch_size_in_joins = @@ -544,7 +557,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { left: left_side_joiner, right: right_side_joiner, column_indices: self.column_indices.clone(), - metrics: StreamJoinMetrics::new(partition, &self.metrics), + metrics: StreamJoinMetrics::new(partition, plan_metrics!(self, state)), graph, left_sorted_filter_expr, right_sorted_filter_expr, @@ -564,7 +577,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { left: left_side_joiner, right: right_side_joiner, column_indices: self.column_indices.clone(), - metrics: StreamJoinMetrics::new(partition, &self.metrics), + metrics: StreamJoinMetrics::new(partition, plan_metrics!(self, state)), graph, left_sorted_filter_expr, right_sorted_filter_expr, diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index 27284bf546bc1..7a57c2dd0b874 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -25,7 +25,7 @@ use crate::joins::{ }; use crate::repartition::RepartitionExec; use crate::test::TestMemoryExec; -use crate::{ExecutionPlan, ExecutionPlanProperties, Partitioning, common}; +use crate::{ExecutionPlan, ExecutionPlanProperties, Partitioning}; use arrow::array::{ ArrayRef, Float64Array, Int32Array, IntervalDayTimeArray, RecordBatch, @@ -107,17 +107,11 @@ pub async fn partitioned_sym_join_with_filter( StreamJoinPartitionMode::Partitioned, )?; - let mut batches = vec![]; - for i in 0..partition_count { - let stream = join.execute(i, Arc::clone(&context))?; - let more_batches = common::collect(stream).await?; - batches.extend( - more_batches - .into_iter() - .filter(|b| b.num_rows() > 0) - .collect::>(), - ); - } + let batches = crate::collect(Arc::new(join), context) + .await? + .into_iter() + .filter(|b| b.num_rows() > 0) + .collect(); Ok(batches) } @@ -154,17 +148,11 @@ pub async fn partitioned_hash_join_with_filter( null_equality, )?); - let mut batches = vec![]; - for i in 0..partition_count { - let stream = join.execute(i, Arc::clone(&context))?; - let more_batches = common::collect(stream).await?; - batches.extend( - more_batches - .into_iter() - .filter(|b| b.num_rows() > 0) - .collect::>(), - ); - } + let batches = crate::collect(join, context) + .await? + .into_iter() + .filter(|b| b.num_rows() > 0) + .collect(); Ok(batches) } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index ec8e154caec91..446cadc2ea980 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -43,10 +43,12 @@ pub use datafusion_physical_expr::{ }; pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; +#[cfg(not(feature = "stateless_plan"))] +pub use crate::execution_plan::execute_input_stream; pub use crate::execution_plan::{ ExecutionPlan, ExecutionPlanProperties, PlanProperties, collect, collect_partitioned, - displayable, execute_input_stream, execute_stream, execute_stream_partitioned, - get_plan_string, with_new_children_if_necessary, + displayable, execute_stream, execute_stream_partitioned, get_plan_string, + with_new_children_if_necessary, }; pub use crate::metrics::Metric; pub use crate::ordering::InputOrderMode; @@ -57,6 +59,7 @@ pub use crate::visitor::{ExecutionPlanVisitor, accept, visit_execution_plan}; pub use crate::work_table::WorkTable; pub use spill::spill_manager::SpillManager; +mod dynamic_filter; mod ordering; mod render_tree; mod topk; @@ -87,6 +90,8 @@ pub mod repartition; pub mod sort_pushdown; pub mod sorts; pub mod spill; +#[cfg(feature = "stateless_plan")] +pub mod state; pub mod stream; pub mod streaming; pub mod tree_node; diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 05d6882821477..075da94231497 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -22,18 +22,22 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use super::metrics::BaselineMetrics; use super::{ DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::execution_plan::{Boundedness, CardinalityEffect}; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{DisplayFormatType, Distribution, ExecutionPlan, Partitioning}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::{Result, assert_eq_or_internal_err, internal_err}; use datafusion_execution::TaskContext; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -48,9 +52,10 @@ pub struct GlobalLimitExec { /// Maximum number of rows to fetch, /// `None` means fetching all rows fetch: Option, + cache: PlanProperties, /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] metrics: ExecutionPlanMetricsSet, - cache: PlanProperties, } impl GlobalLimitExec { @@ -61,8 +66,9 @@ impl GlobalLimitExec { input, skip, fetch, - metrics: ExecutionPlanMetricsSet::new(), cache, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), } } @@ -164,7 +170,13 @@ impl ExecutionPlan for GlobalLimitExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + trace!("Start GlobalLimitExec::execute for partition: {partition}"); // GlobalLimitExec has a single output partition assert_eq_or_internal_err!( @@ -180,8 +192,9 @@ impl ExecutionPlan for GlobalLimitExec { "GlobalLimitExec requires a single input partition" ); - let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); - let stream = self.input.execute(0, context)?; + let baseline_metrics = + BaselineMetrics::new(plan_metrics!(self, state), partition); + let stream = execute_input!(0, self.input, 0, context, state)?; Ok(Box::pin(LimitStream::new( stream, self.skip, @@ -190,6 +203,7 @@ impl ExecutionPlan for GlobalLimitExec { ))) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -220,9 +234,10 @@ pub struct LocalLimitExec { input: Arc, /// Maximum number of rows to return fetch: usize, + cache: PlanProperties, /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] metrics: ExecutionPlanMetricsSet, - cache: PlanProperties, } impl LocalLimitExec { @@ -232,8 +247,9 @@ impl LocalLimitExec { Self { input, fetch, - metrics: ExecutionPlanMetricsSet::new(), cache, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), } } @@ -319,15 +335,22 @@ impl ExecutionPlan for LocalLimitExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + trace!( "Start LocalLimitExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id() ); - let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); - let stream = self.input.execute(partition, context)?; + let baseline_metrics = + BaselineMetrics::new(plan_metrics!(self, state), partition); + let stream = execute_input!(0, self.input, partition, context, state)?; Ok(Box::pin(LimitStream::new( stream, 0, @@ -336,6 +359,7 @@ impl ExecutionPlan for LocalLimitExec { ))) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -493,12 +517,14 @@ mod tests { use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::common::collect; + use crate::execution_plan::execute_plan; use crate::test; use crate::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use arrow::array::RecordBatchOptions; use arrow::datatypes::Schema; use datafusion_common::stats::Precision; + use datafusion_execution::metrics::ExecutionPlanMetricsSet; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr::expressions::col; @@ -516,7 +542,7 @@ mod tests { GlobalLimitExec::new(Arc::new(CoalescePartitionsExec::new(csv)), 0, Some(7)); // The result should contain 4 batches (one per input partition) - let iter = limit.execute(0, task_ctx)?; + let iter = execute_plan(Arc::new(limit), 0, task_ctx)?; let batches = collect(iter).await?; // There should be a total of 100 rows @@ -632,7 +658,7 @@ mod tests { GlobalLimitExec::new(Arc::new(CoalescePartitionsExec::new(csv)), skip, fetch); // The result should contain 4 batches (one per input partition) - let iter = offset.execute(0, task_ctx)?; + let iter = execute_plan(Arc::new(offset), 0, task_ctx)?; let batches = collect(iter).await?; Ok(batches.iter().map(|batch| batch.num_rows()).sum()) } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 65a3fe575e178..81b9904ca74c5 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -24,7 +24,9 @@ use std::task::{Context, Poll}; use crate::coop::cooperative; use crate::execution_plan::{Boundedness, EmissionType, SchedulingType}; -use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use crate::metrics::BaselineMetrics; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -35,6 +37,8 @@ use arrow::datatypes::SchemaRef; use datafusion_common::{Result, assert_eq_or_internal_err, assert_or_internal_err}; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::MemoryReservation; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; @@ -160,6 +164,7 @@ pub struct LazyMemoryExec { /// Plan properties cache storing equivalence properties, partitioning, and execution mode cache: PlanProperties, /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] metrics: ExecutionPlanMetricsSet, } @@ -204,6 +209,7 @@ impl LazyMemoryExec { projection: None, batch_generators: generators, cache, + #[cfg(not(feature = "stateless_plan"))] metrics: ExecutionPlanMetricsSet::new(), }) } @@ -326,7 +332,13 @@ impl ExecutionPlan for LazyMemoryExec { &self, partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::plan_metrics; + assert_or_internal_err!( partition < self.batch_generators.len(), "Invalid partition {} for LazyMemoryExec with {} partitions", @@ -334,7 +346,8 @@ impl ExecutionPlan for LazyMemoryExec { self.batch_generators.len() ); - let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + let baseline_metrics = + BaselineMetrics::new(plan_metrics!(self, state), partition); let stream = LazyMemoryStream { schema: Arc::clone(&self.schema), @@ -345,6 +358,7 @@ impl ExecutionPlan for LazyMemoryExec { Ok(Box::pin(cooperative(stream))) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -408,6 +422,7 @@ impl RecordBatchStream for LazyMemoryStream { mod lazy_memory_tests { use super::*; use crate::common::collect; + use crate::execution_plan::{execute_plan, execute_plan_and_get_metrics_of}; use arrow::array::Int64Array; use arrow::datatypes::{DataType, Field, Schema}; use futures::StreamExt; @@ -470,7 +485,7 @@ mod lazy_memory_tests { assert_eq!(exec.schema().field(0).name(), "a"); // Test execution - let stream = exec.execute(0, Arc::new(TaskContext::default()))?; + let stream = execute_plan(Arc::new(exec), 0, Arc::new(TaskContext::default()))?; let batches: Vec<_> = stream.collect::>().await; assert_eq!(batches.len(), 3); @@ -517,7 +532,7 @@ mod lazy_memory_tests { LazyMemoryExec::try_new(schema, vec![Arc::new(RwLock::new(generator))])?; // Test invalid partition - let result = exec.execute(1, Arc::new(TaskContext::default())); + let result = execute_plan(Arc::new(exec), 1, Arc::new(TaskContext::default())); // partition is 0-indexed, so there only should be partition 0 assert!(matches!( @@ -547,15 +562,18 @@ mod lazy_memory_tests { schema: Arc::clone(&schema), }; - let exec = - LazyMemoryExec::try_new(schema, vec![Arc::new(RwLock::new(generator))])?; + let exec: Arc = Arc::new(LazyMemoryExec::try_new( + schema, + vec![Arc::new(RwLock::new(generator))], + )?); let task_ctx = Arc::new(TaskContext::default()); - let stream = exec.execute(0, task_ctx)?; + let (stream, metrics) = + execute_plan_and_get_metrics_of(Arc::clone(&exec), &exec, 0, task_ctx)?; let batches = collect(stream).await?; // Verify metrics exist with actual expected numbers - let metrics = exec.metrics().unwrap(); + let metrics = metrics.unwrap(); // Count actual rows returned let actual_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 4d00b73cff39c..fc37d348994a1 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -23,6 +23,8 @@ use std::sync::Arc; use crate::coop::cooperative; use crate::execution_plan::{Boundedness, EmissionType, SchedulingType}; use crate::memory::MemoryStream; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, common, @@ -151,6 +153,7 @@ impl ExecutionPlan for PlaceholderRowExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { trace!( "Start PlaceholderRowExec::execute for partition {} of context session_id {} and task_id {:?}", @@ -195,6 +198,7 @@ impl ExecutionPlan for PlaceholderRowExec { #[cfg(test)] mod tests { use super::*; + use crate::execution_plan::execute_plan; use crate::test; use crate::with_new_children_if_necessary; @@ -222,11 +226,16 @@ mod tests { async fn invalid_execute() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let schema = test::aggr_test_schema(); - let placeholder = PlaceholderRowExec::new(schema); + let placeholder: Arc = + Arc::new(PlaceholderRowExec::new(schema)); // Ask for the wrong partition - assert!(placeholder.execute(1, Arc::clone(&task_ctx)).is_err()); - assert!(placeholder.execute(20, task_ctx).is_err()); + assert!( + execute_plan(Arc::clone(&placeholder), 1, Arc::clone(&task_ctx)).is_err() + ); + assert!( + execute_plan(Arc::clone(&placeholder), 20, Arc::clone(&task_ctx)).is_err() + ); Ok(()) } @@ -236,7 +245,7 @@ mod tests { let schema = test::aggr_test_schema(); let placeholder = PlaceholderRowExec::new(schema); - let iter = placeholder.execute(0, task_ctx)?; + let iter = execute_plan(Arc::new(placeholder), 0, task_ctx)?; let batches = common::collect(iter).await?; // Should have one item @@ -250,10 +259,11 @@ mod tests { let task_ctx = Arc::new(TaskContext::default()); let schema = test::aggr_test_schema(); let partitions = 3; - let placeholder = PlaceholderRowExec::new(schema).with_partitions(partitions); + let placeholder: Arc = + Arc::new(PlaceholderRowExec::new(schema).with_partitions(partitions)); for n in 0..partitions { - let iter = placeholder.execute(n, Arc::clone(&task_ctx))?; + let iter = execute_plan(Arc::clone(&placeholder), n, Arc::clone(&task_ctx))?; let batches = common::collect(iter).await?; // Should have one item diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index a56e9272f119e..74c77e5f01bb3 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -21,7 +21,7 @@ //! projection expressions. `SELECT` without `FROM` will only evaluate expressions. use super::expressions::{Column, Literal}; -use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use super::metrics::BaselineMetrics; use super::{ DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -32,6 +32,8 @@ use crate::filter_pushdown::{ FilterPushdownPropagation, }; use crate::joins::utils::{ColumnIndex, JoinFilter, JoinOn, JoinOnRef}; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{DisplayFormatType, ExecutionPlan, PhysicalExpr}; use std::any::Any; use std::collections::HashMap; @@ -47,6 +49,8 @@ use datafusion_common::tree_node::{ }; use datafusion_common::{JoinSide, Result, internal_err}; use datafusion_execution::TaskContext; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::projection::Projector; use datafusion_physical_expr::utils::collect_columns; @@ -72,10 +76,11 @@ pub struct ProjectionExec { projector: Projector, /// The input plan input: Arc, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } impl ProjectionExec { @@ -149,8 +154,9 @@ impl ProjectionExec { Ok(Self { projector, input, - metrics: ExecutionPlanMetricsSet::new(), cache, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), }) } @@ -286,7 +292,13 @@ impl ExecutionPlan for ProjectionExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + trace!( "Start ProjectionExec::execute for partition {} of context session_id {} and task_id {:?}", partition, @@ -295,11 +307,12 @@ impl ExecutionPlan for ProjectionExec { ); Ok(Box::pin(ProjectionStream::new( self.projector.clone(), - self.input.execute(partition, context)?, - BaselineMetrics::new(&self.metrics, partition), + execute_input!(0, self.input, partition, context, state)?, + BaselineMetrics::new(plan_metrics!(self, state), partition), )?)) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -1002,6 +1015,7 @@ mod tests { use std::sync::Arc; use crate::common::collect; + use crate::execution_plan::execute_plan; use crate::test; use crate::test::exec::StatisticsExec; @@ -1091,10 +1105,11 @@ mod tests { let task_ctx = Arc::new(TaskContext::default()); let exec = test::scan_partitioned(1); - let expected = collect(exec.execute(0, Arc::clone(&task_ctx))?).await?; + let expected = + collect(execute_plan(Arc::clone(&exec), 0, Arc::clone(&task_ctx))?).await?; let projection = ProjectionExec::try_new(vec![] as Vec, exec)?; - let stream = projection.execute(0, Arc::clone(&task_ctx))?; + let stream = execute_plan(Arc::new(projection), 0, Arc::clone(&task_ctx))?; let output = collect(stream).await?; assert_eq!(output.len(), expected.len()); diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index 3e7c75b0c8e85..1ec0fd673955f 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -25,22 +25,30 @@ use super::work_table::{ReservedBatches, WorkTable}; use crate::aggregates::group_values::{GroupValues, new_group_values}; use crate::aggregates::order::GroupOrdering; use crate::execution_plan::{Boundedness, EmissionType}; -use crate::metrics::{ - BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet, RecordOutput, -}; +use crate::metrics::{BaselineMetrics, RecordOutput}; use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; +#[cfg(feature = "stateless_plan")] +use crate::{ + state::{PlanState, PlanStateNode}, + work_table::WorkTableExec, +}; use arrow::array::{BooleanArray, BooleanBuilder}; use arrow::compute::filter_record_batch; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{Result, internal_datafusion_err, not_impl_err}; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; +#[cfg(not(feature = "stateless_plan"))] +use { + datafusion_common::tree_node::TransformedResult, + datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}, +}; use futures::{Stream, StreamExt, ready}; @@ -63,18 +71,43 @@ use futures::{Stream, StreamExt, ready}; pub struct RecursiveQueryExec { /// Name of the query handler name: String, - /// The working table of cte - work_table: Arc, /// The base part (static term) static_term: Arc, /// The dynamic part (recursive term) recursive_term: Arc, /// Distinction is_distinct: bool, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// Plan state. + #[cfg(not(feature = "stateless_plan"))] + state: RecursiveQueryExecState, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, +} + +#[derive(Debug, Clone)] +struct RecursiveQueryExecState { + /// The working table of cte. + work_table: Arc, +} + +impl RecursiveQueryExecState { + fn new(work_table: Arc) -> Self { + Self { work_table } + } +} + +#[cfg(feature = "stateless_plan")] +impl PlanState for RecursiveQueryExecState { + fn as_any(&self) -> &dyn Any { + self + } + + fn work_table(&self) -> Option> { + Some(Arc::clone(&self.work_table)) + } } impl RecursiveQueryExec { @@ -86,18 +119,27 @@ impl RecursiveQueryExec { is_distinct: bool, ) -> Result { // Each recursive query needs its own work table + #[cfg(not(feature = "stateless_plan"))] let work_table = Arc::new(WorkTable::new()); + // Use the same work table for both the WorkTableExec and the recursive term + #[cfg(not(feature = "stateless_plan"))] let recursive_term = assign_work_table(recursive_term, &work_table)?; + + #[cfg(feature = "stateless_plan")] + ensure_input_is_supported(Arc::clone(&recursive_term))?; + let cache = Self::compute_properties(static_term.schema()); Ok(RecursiveQueryExec { name, static_term, recursive_term, is_distinct, - work_table, - metrics: ExecutionPlanMetricsSet::new(), cache, + #[cfg(not(feature = "stateless_plan"))] + state: RecursiveQueryExecState::new(work_table), + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), }) } @@ -185,7 +227,13 @@ impl ExecutionPlan for RecursiveQueryExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + // TODO: we might be able to handle multiple partitions in the future. if partition != 0 { return Err(internal_datafusion_err!( @@ -193,18 +241,31 @@ impl ExecutionPlan for RecursiveQueryExec { )); } - let static_stream = self.static_term.execute(partition, Arc::clone(&context))?; - let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + #[cfg(feature = "stateless_plan")] + let exec_state = state.get_or_init_state(|| { + RecursiveQueryExecState::new(Arc::new(WorkTable::new())) + }); + + #[cfg(not(feature = "stateless_plan"))] + let exec_state = &self.state; + + let static_stream = + execute_input!(0, self.static_term, partition, Arc::clone(&context), state)?; + let baseline_metrics = + BaselineMetrics::new(plan_metrics!(self, state), partition); Ok(Box::pin(RecursiveQueryStream::new( context, - Arc::clone(&self.work_table), + Arc::clone(&exec_state.work_table), Arc::clone(&self.recursive_term), static_stream, self.is_distinct, baseline_metrics, + #[cfg(feature = "stateless_plan")] + Arc::clone(&state.plan_node), )?)) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -277,6 +338,9 @@ struct RecursiveQueryStream { distinct_deduplicator: Option, /// Metrics. baseline_metrics: BaselineMetrics, + /// [`RecursiveQueryExec`] plan reference. + #[cfg(feature = "stateless_plan")] + parent_plan: Arc, } impl RecursiveQueryStream { @@ -288,6 +352,7 @@ impl RecursiveQueryStream { static_stream: SendableRecordBatchStream, is_distinct: bool, baseline_metrics: BaselineMetrics, + #[cfg(feature = "stateless_plan")] parent_plan: Arc, ) -> Result { let schema = static_stream.schema(); let reservation = @@ -306,6 +371,8 @@ impl RecursiveQueryStream { reservation, distinct_deduplicator, baseline_metrics, + #[cfg(feature = "stateless_plan")] + parent_plan, }) } @@ -356,13 +423,61 @@ impl RecursiveQueryStream { // Downstream plans should not expect any partitioning. let partition = 0; - let recursive_plan = reset_plan_states(Arc::clone(&self.recursive_term))?; - self.recursive_stream = - Some(recursive_plan.execute(partition, Arc::clone(&self.task_context))?); + #[cfg(feature = "stateless_plan")] + { + // Initialize a new state for the recursive term. This way we also pass + // work table into the recursive term. + // TODO: figure out how to merge metrics. + use crate::state::WithPlanStateNode; + let state = PlanStateNode::new_root_arc(Arc::clone(&self.parent_plan)); + state.get_or_init_state(|| RecursiveQueryExecState { + work_table: Arc::clone(&self.work_table), + }); + let raw_recursive_stream = self.recursive_term.execute( + partition, + Arc::clone(&self.task_context), + &state.child_state(1), + )?; + self.recursive_stream = Some(Box::pin(WithPlanStateNode::new( + raw_recursive_stream, + state, + ))); + } + + #[cfg(not(feature = "stateless_plan"))] + { + let recursive_plan = reset_plan_states(Arc::clone(&self.recursive_term))?; + self.recursive_stream = + Some(recursive_plan.execute(partition, Arc::clone(&self.task_context))?); + } + self.poll_next(cx) } } +#[cfg(feature = "stateless_plan")] +fn ensure_input_is_supported(plan: Arc) -> Result<()> { + let mut work_table_refs = 0; + plan.transform_down(|plan| { + if plan.as_any().is::() { + if work_table_refs > 0 { + not_impl_err!( + "Multiple recursive references to the same CTE are not supported" + ) + } else { + work_table_refs += 1; + Ok(Transformed::no(plan)) + } + } else if plan.as_any().is::() { + not_impl_err!("Recursive queries cannot be nested") + } else { + Ok(Transformed::no(plan)) + } + }) + .map(|_| ()) +} + +#[cfg(not(feature = "stateless_plan"))] fn assign_work_table( plan: Arc, work_table: &Arc, @@ -395,6 +510,7 @@ fn assign_work_table( /// An example is `CrossJoinExec`, which loads the left table into memory and stores it in the plan. /// However, if the data of the left table is derived from the work table, it will become outdated /// as the work table changes. When the next iteration executes this plan again, we must clear the left table. +#[cfg(not(feature = "stateless_plan"))] fn reset_plan_states(plan: Arc) -> Result> { plan.transform_up(|plan| { let new_plan = Arc::clone(&plan).reset_state()?; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 1efdaaabc7d6a..0a7514082437c 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -26,7 +26,7 @@ use std::task::{Context, Poll}; use std::{any::Any, vec}; use super::common::SharedMemoryReservation; -use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; +use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder}; use super::{ DisplayAs, ExecutionPlanProperties, RecordBatchStream, SendableRecordBatchStream, }; @@ -38,6 +38,8 @@ use crate::projection::{ProjectionExec, all_columns, make_with_child, update_exp use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::spill_manager::SpillManager; use crate::spill::spill_pool::{self, SpillPoolWriter}; +#[cfg(feature = "stateless_plan")] +use crate::state::{PlanState, PlanStateNode}; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; @@ -54,6 +56,8 @@ use datafusion_common::{Result, not_impl_err}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::MemoryConsumer; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::MetricsSet; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; @@ -201,7 +205,7 @@ impl Debug for ConsumingInputStreamsState { /// Inner state of [`RepartitionExec`]. #[derive(Default)] -enum RepartitionExecState { +enum RepartitionExecStateInner { /// Not initialized yet. This is the default state stored in the RepartitionExec node /// upon instantiation. #[default] @@ -215,45 +219,54 @@ enum RepartitionExecState { ConsumingInputStreams(ConsumingInputStreamsState), } -impl Debug for RepartitionExecState { +impl Debug for RepartitionExecStateInner { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { match self { - RepartitionExecState::NotInitialized => write!(f, "NotInitialized"), - RepartitionExecState::InputStreamsInitialized(v) => { + Self::NotInitialized => write!(f, "NotInitialized"), + Self::InputStreamsInitialized(v) => { write!(f, "InputStreamsInitialized({:?})", v.len()) } - RepartitionExecState::ConsumingInputStreams(v) => { + Self::ConsumingInputStreams(v) => { write!(f, "ConsumingInputStreams({v:?})") } } } } -impl RepartitionExecState { +impl RepartitionExecStateInner { fn ensure_input_streams_initialized( &mut self, input: &Arc, metrics: &ExecutionPlanMetricsSet, output_partitions: usize, ctx: &Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result<()> { - if !matches!(self, RepartitionExecState::NotInitialized) { + if !matches!(self, Self::NotInitialized) { return Ok(()); } let num_input_partitions = input.output_partitioning().partition_count(); let mut streams_and_metrics = Vec::with_capacity(num_input_partitions); + #[cfg(feature = "stateless_plan")] + let child_state = state.child_state(0); for i in 0..num_input_partitions { let metrics = RepartitionMetrics::new(i, output_partitions, metrics); let timer = metrics.fetch_time.timer(); + + #[cfg(not(feature = "stateless_plan"))] let stream = input.execute(i, Arc::clone(ctx))?; + + #[cfg(feature = "stateless_plan")] + let stream = input.execute(i, Arc::clone(ctx), &child_state)?; + timer.done(); streams_and_metrics.push((stream, metrics)); } - *self = RepartitionExecState::InputStreamsInitialized(streams_and_metrics); + *self = Self::InputStreamsInitialized(streams_and_metrics); Ok(()) } @@ -267,16 +280,19 @@ impl RepartitionExecState { name: &str, context: &Arc, spill_manager: SpillManager, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result<&mut ConsumingInputStreamsState> { let streams_and_metrics = match self { - RepartitionExecState::NotInitialized => { + Self::NotInitialized => { self.ensure_input_streams_initialized( input, metrics, partitioning.partition_count(), context, + #[cfg(feature = "stateless_plan")] + state, )?; - let RepartitionExecState::InputStreamsInitialized(value) = self else { + let Self::InputStreamsInitialized(value) = self else { // This cannot happen, as ensure_input_streams_initialized() was just called, // but the compiler does not know. return internal_err!( @@ -285,8 +301,8 @@ impl RepartitionExecState { }; value } - RepartitionExecState::ConsumingInputStreams(value) => return Ok(value), - RepartitionExecState::InputStreamsInitialized(value) => value, + Self::ConsumingInputStreams(value) => return Ok(value), + Self::InputStreamsInitialized(value) => value, }; let num_input_partitions = streams_and_metrics.len(); @@ -404,12 +420,21 @@ impl RepartitionExecState { abort_helper: Arc::new(spawned_tasks), }); match self { - RepartitionExecState::ConsumingInputStreams(value) => Ok(value), + Self::ConsumingInputStreams(value) => Ok(value), _ => unreachable!(), } } } +type RepartitionExecState = Arc>; + +#[cfg(feature = "stateless_plan")] +impl PlanState for RepartitionExecState { + fn as_any(&self) -> &dyn Any { + self + } +} + /// A utility that can be used to partition batches based on [`Partitioning`] pub struct BatchPartitioner { state: BatchPartitionerState, @@ -678,16 +703,18 @@ impl BatchPartitioner { pub struct RepartitionExec { /// Input execution plan input: Arc, - /// Inner state that is initialized when the parent calls .execute() on this node - /// and consumed as soon as the parent starts consuming this node. - state: Arc>, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// Boolean flag to decide whether to preserve ordering. If true means /// `SortPreservingRepartitionExec`, false means `RepartitionExec`. preserve_order: bool, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + #[cfg(not(feature = "stateless_plan"))] + /// Inner state that is initialized when the parent calls .execute() on this node + /// and consumed as soon as the parent starts consuming this node. + state: RepartitionExecState, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } #[derive(Debug, Clone)] @@ -849,18 +876,24 @@ impl ExecutionPlan for RepartitionExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::plan_metrics; + trace!( "Start {}::execute for partition: {}", self.name(), partition ); - let spill_metrics = SpillMetrics::new(&self.metrics, partition); + let spill_metrics = SpillMetrics::new(plan_metrics!(self, state), partition); let input = Arc::clone(&self.input); let partitioning = self.partitioning().clone(); - let metrics = self.metrics.clone(); + let metrics = plan_metrics!(self, state).clone(); let preserve_order = self.sort_exprs().is_some(); let name = self.name().to_owned(); let schema = self.schema(); @@ -875,24 +908,35 @@ impl ExecutionPlan for RepartitionExec { // Get existing ordering to use for merging let sort_exprs = self.sort_exprs().cloned(); - let state = Arc::clone(&self.state); - if let Some(mut state) = state.try_lock() { - state.ensure_input_streams_initialized( + #[cfg(feature = "stateless_plan")] + let exec_state = Arc::clone(state.get_or_init_state(|| { + Arc::new(Mutex::new(RepartitionExecStateInner::default())) + })); + + #[cfg(not(feature = "stateless_plan"))] + let exec_state = Arc::clone(&self.state); + + if let Some(mut exec_state) = exec_state.try_lock() { + exec_state.ensure_input_streams_initialized( &input, &metrics, partitioning.partition_count(), &context, + #[cfg(feature = "stateless_plan")] + state, )?; } let num_input_partitions = input.output_partitioning().partition_count(); + #[cfg(feature = "stateless_plan")] + let state = Arc::clone(state); let stream = futures::stream::once(async move { // lock scope let (rx, reservation, spill_readers, abort_helper) = { // lock mutexes - let mut state = state.lock(); - let state = state.consume_input_streams( + let mut exec_state = exec_state.lock(); + let exec_state = exec_state.consume_input_streams( &input, &metrics, &partitioning, @@ -900,6 +944,8 @@ impl ExecutionPlan for RepartitionExec { &name, &context, spill_manager.clone(), + #[cfg(feature = "stateless_plan")] + &state, )?; // now return stream for the specified *output* partition which will @@ -909,7 +955,7 @@ impl ExecutionPlan for RepartitionExec { reservation, spill_readers, .. - } = state + } = exec_state .channels .remove(&partition) .expect("partition not used yet"); @@ -918,7 +964,7 @@ impl ExecutionPlan for RepartitionExec { rx, reservation, spill_readers, - Arc::clone(&state.abort_helper), + Arc::clone(&exec_state.abort_helper), ) }; @@ -990,6 +1036,7 @@ impl ExecutionPlan for RepartitionExec { Ok(Box::pin(stream)) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -1136,10 +1183,12 @@ impl ExecutionPlan for RepartitionExec { }; Ok(Some(Arc::new(Self { input: Arc::clone(&self.input), - state: Arc::clone(&self.state), - metrics: self.metrics.clone(), preserve_order: self.preserve_order, cache: new_properties, + #[cfg(not(feature = "stateless_plan"))] + state: Arc::clone(&self.state), + #[cfg(not(feature = "stateless_plan"))] + metrics: self.metrics.clone(), }))) } } @@ -1156,10 +1205,12 @@ impl RepartitionExec { let cache = Self::compute_properties(&input, partitioning, preserve_order); Ok(RepartitionExec { input, - state: Default::default(), - metrics: ExecutionPlanMetricsSet::new(), preserve_order, cache, + #[cfg(not(feature = "stateless_plan"))] + state: Default::default(), + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), }) } @@ -1641,7 +1692,9 @@ mod tests { use std::collections::HashSet; use super::*; + use crate::execution_plan::execute_plan; use crate::test::TestMemoryExec; + use crate::test::{collect_counting_rows, collect_partitions, collect_with}; use crate::{ test::{ assert_is_pending, @@ -1763,15 +1816,15 @@ mod tests { // create physical plan let exec = TestMemoryExec::try_new_exec(&partitions, Arc::clone(&schema), None)?; - let exec = RepartitionExec::try_new(exec, partitioning)?; + let exec: Arc = + Arc::new(RepartitionExec::try_new(exec, partitioning)?); + + collect_with(exec, task_ctx, |_, batch| { + assert_eq!(200, batch?.num_rows()); + Ok(()) + }) + .await?; - for i in 0..exec.partitioning().partition_count() { - let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; - while let Some(result) = stream.next().await { - let batch = result?; - assert_eq!(200, batch.num_rows()); - } - } Ok(()) } @@ -1788,19 +1841,11 @@ mod tests { // create physical plan let exec = TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(schema), None)?; - let exec = RepartitionExec::try_new(exec, partitioning)?; + let exec: Arc = + Arc::new(RepartitionExec::try_new(exec, partitioning)?); // execute and collect results - let mut output_partitions = vec![]; - for i in 0..exec.partitioning().partition_count() { - // execute this *output* partition and collect all batches - let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; - let mut batches = vec![]; - while let Some(result) = stream.next().await { - batches.push(result?); - } - output_partitions.push(batches); - } + let (output_partitions, _) = collect_partitions(exec, task_ctx).await?; Ok(output_partitions) } @@ -1847,7 +1892,7 @@ mod tests { // returned and no results produced let partitioning = Partitioning::UnknownPartitioning(1); let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); - let output_stream = exec.execute(0, task_ctx).unwrap(); + let output_stream = execute_plan(Arc::new(exec), 0, task_ctx).unwrap(); // Expect that an error is returned let result_string = crate::common::collect(output_stream) @@ -1872,7 +1917,10 @@ mod tests { let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); // Expect that an error is returned - let result_string = exec.execute(0, task_ctx).err().unwrap().to_string(); + let result_string = execute_plan(Arc::new(exec), 0, task_ctx) + .err() + .unwrap() + .to_string(); assert!( result_string.contains("ErrorExec, unsurprisingly, errored in partition 0"), @@ -1900,7 +1948,7 @@ mod tests { // Note: this should pass (the stream can be created) but the // error when the input is executed should get passed back - let output_stream = exec.execute(0, task_ctx).unwrap(); + let output_stream = execute_plan(Arc::new(exec), 0, task_ctx).unwrap(); // Expect that an error is returned let result_string = crate::common::collect(output_stream) @@ -1948,7 +1996,7 @@ mod tests { +------------------+ "); - let output_stream = exec.execute(0, task_ctx).unwrap(); + let output_stream = execute_plan(Arc::new(exec), 0, task_ctx).unwrap(); let batches = crate::common::collect(output_stream).await.unwrap(); assert_snapshot!(batches_to_sort_string(&batches), @r" @@ -1972,14 +2020,18 @@ mod tests { let input = Arc::new(make_barrier_exec()); // partition into two output streams - let exec = RepartitionExec::try_new( - Arc::clone(&input) as Arc, - partitioning, - ) - .unwrap(); + let exec: Arc = Arc::new( + RepartitionExec::try_new( + Arc::clone(&input) as Arc, + partitioning, + ) + .unwrap(), + ); - let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap(); - let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap(); + let output_stream0 = + execute_plan(Arc::clone(&exec), 0, Arc::clone(&task_ctx)).unwrap(); + let output_stream1 = + execute_plan(Arc::clone(&exec), 1, Arc::clone(&task_ctx)).unwrap(); // now, purposely drop output stream 0 // *before* any outputs are produced @@ -2027,7 +2079,8 @@ mod tests { partitioning.clone(), ) .unwrap(); - let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap(); + let output_stream1 = + execute_plan(Arc::new(exec), 1, Arc::clone(&task_ctx)).unwrap(); let mut background_task = JoinSet::new(); background_task.spawn(async move { input.wait().await; @@ -2047,13 +2100,17 @@ mod tests { // Now do the same but dropping the stream before waiting for the barrier let input = Arc::new(make_barrier_exec()); - let exec = RepartitionExec::try_new( - Arc::clone(&input) as Arc, - partitioning, - ) - .unwrap(); - let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap(); - let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap(); + let exec: Arc = Arc::new( + RepartitionExec::try_new( + Arc::clone(&input) as Arc, + partitioning, + ) + .unwrap(), + ); + let output_stream0 = + execute_plan(Arc::clone(&exec), 0, Arc::clone(&task_ctx)).unwrap(); + let output_stream1 = + execute_plan(Arc::clone(&exec), 1, Arc::clone(&task_ctx)).unwrap(); // now, purposely drop output stream 0 // *before* any outputs are produced drop(output_stream0); @@ -2157,10 +2214,11 @@ mod tests { ); let schema = batch.schema(); let input = MockExec::new(vec![Ok(batch)], schema); - let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); - let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap(); + let exec: Arc = + Arc::new(RepartitionExec::try_new(Arc::new(input), partitioning).unwrap()); + let output_stream0 = execute_plan(Arc::clone(&exec), 0, Arc::clone(&task_ctx))?; let batch0 = crate::common::collect(output_stream0).await.unwrap(); - let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap(); + let output_stream1 = execute_plan(Arc::clone(&exec), 1, Arc::clone(&task_ctx))?; let batch1 = crate::common::collect(output_stream1).await.unwrap(); assert!(batch0.is_empty() || batch1.is_empty()); Ok(()) @@ -2185,23 +2243,17 @@ mod tests { // create physical plan let exec = TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(&schema), None)?; - let exec = RepartitionExec::try_new(exec, partitioning)?; + let exec: Arc = + Arc::new(RepartitionExec::try_new(exec, partitioning)?); // Collect all partitions - should succeed by spilling to disk - let mut total_rows = 0; - for i in 0..exec.partitioning().partition_count() { - let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; - while let Some(result) = stream.next().await { - let batch = result?; - total_rows += batch.num_rows(); - } - } + let (total_rows, metrics) = collect_counting_rows(exec, task_ctx).await?; // Verify we got all the data (50 batches * 8 rows each) assert_eq!(total_rows, 50 * 8); // Verify spilling metrics to confirm spilling actually happened - let metrics = exec.metrics().unwrap(); + assert!( metrics.spill_count().unwrap() > 0, "Expected spill_count > 0, but got {:?}", @@ -2251,20 +2303,14 @@ mod tests { let exec = RepartitionExec::try_new(exec, partitioning)?; // Collect all partitions - should succeed with partial spilling - let mut total_rows = 0; - for i in 0..exec.partitioning().partition_count() { - let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; - while let Some(result) = stream.next().await { - let batch = result?; - total_rows += batch.num_rows(); - } - } + let (total_rows, metrics) = + collect_counting_rows(Arc::new(exec), task_ctx).await?; // Verify we got all the data (50 batches * 8 rows each) assert_eq!(total_rows, 50 * 8); // Verify partial spilling metrics - let metrics = exec.metrics().unwrap(); + // Verify spilling metrics to confirm spilling actually happened let spill_count = metrics.spill_count().unwrap(); let spilled_rows = metrics.spilled_rows().unwrap(); let spilled_bytes = metrics.spilled_bytes().unwrap(); @@ -2316,20 +2362,12 @@ mod tests { let exec = RepartitionExec::try_new(exec, partitioning)?; // Collect all partitions - should succeed without spilling - let mut total_rows = 0; - for i in 0..exec.partitioning().partition_count() { - let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; - while let Some(result) = stream.next().await { - let batch = result?; - total_rows += batch.num_rows(); - } - } - + let (total_rows, metrics) = + collect_counting_rows(Arc::new(exec), task_ctx).await?; // Verify we got all the data (50 batches * 8 rows each) assert_eq!(total_rows, 50 * 8); // Verify no spilling occurred - let metrics = exec.metrics().unwrap(); assert_eq!( metrics.spill_count(), Some(0), @@ -2378,19 +2416,20 @@ mod tests { // create physical plan let exec = TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(&schema), None)?; - let exec = RepartitionExec::try_new(exec, partitioning)?; + let exec: Arc = + Arc::new(RepartitionExec::try_new(exec, partitioning)?); // Attempt to execute - should fail with ResourcesExhausted error - for i in 0..exec.partitioning().partition_count() { - let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; - let err = stream.next().await.unwrap().unwrap_err(); + collect_with(exec, task_ctx, |_, result| { + let err = result.unwrap_err(); let err = err.find_root(); assert!( matches!(err, DataFusionError::ResourcesExhausted(_)), "Wrong error type: {err}", ); - } - + Ok(()) + }) + .await?; Ok(()) } @@ -2456,19 +2495,9 @@ mod tests { let exec = RepartitionExec::try_new(exec, partitioning)?; // Collect all output partitions - let mut all_batches = Vec::new(); - for i in 0..exec.partitioning().partition_count() { - let mut partition_batches = Vec::new(); - let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; - while let Some(result) = stream.next().await { - let batch = result?; - partition_batches.push(batch); - } - all_batches.push(partition_batches); - } + let (all_batches, metrics) = collect_partitions(Arc::new(exec), task_ctx).await?; // Verify spilling occurred - let metrics = exec.metrics().unwrap(); assert!( metrics.spill_count().unwrap() > 0, "Expected spilling to occur, but spill_count = 0" @@ -2511,6 +2540,7 @@ mod test { use super::*; use crate::test::TestMemoryExec; + use crate::test::collect_with; use crate::union::UnionExec; use datafusion_physical_expr::expressions::col; @@ -2634,15 +2664,11 @@ mod test { .with_preserve_order(); let mut batches = vec![]; - - // Collect all partitions - should succeed by spilling to disk - for i in 0..exec.partitioning().partition_count() { - let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; - while let Some(result) = stream.next().await { - let batch = result?; - batches.push(batch); - } - } + let metrics = collect_with(Arc::new(exec), task_ctx, |_, batch| { + batches.push(batch?); + Ok(()) + }) + .await?; #[rustfmt::skip] let expected = [ @@ -2686,7 +2712,6 @@ mod test { // - We spill data during the repartitioning phase // - We may also spill during the final merge sort let all_batches = [batch1, batch2, batch3, batch4, batch5, batch6]; - let metrics = exec.metrics().unwrap(); assert!( metrics.spill_count().unwrap() > input_partitions.len(), "Expected spill_count > {} for order-preserving repartition, but got {:?}", @@ -2747,13 +2772,22 @@ mod test { let exec = Arc::new(TestMemoryExec::update_cache(&exec)); // Hash partition into 2 partitions by column c0 let hash_expr = col("c0", &schema)?; - let exec = - RepartitionExec::try_new(exec, Partitioning::Hash(vec![hash_expr], 2))?; + let exec: Arc = Arc::new(RepartitionExec::try_new( + exec, + Partitioning::Hash(vec![hash_expr], 2), + )?); + + #[cfg(feature = "stateless_plan")] + let state = PlanStateNode::new_root_arc(Arc::clone(&exec)); // Collect all partitions concurrently using JoinSet - this prevents deadlock // where the distribution channel gate closes when all output channels are full let mut join_set = tokio::task::JoinSet::new(); - for i in 0..exec.partitioning().partition_count() { + for i in 0..exec.properties().partitioning.partition_count() { + #[cfg(feature = "stateless_plan")] + let stream = exec.execute(i, Arc::clone(&task_ctx), &state)?; + + #[cfg(not(feature = "stateless_plan"))] let stream = exec.execute(i, Arc::clone(&task_ctx))?; join_set.spawn(async move { let mut count = 0; @@ -2778,7 +2812,12 @@ mod test { assert_eq!(total_rows, expected_rows); // Verify metrics are available + #[cfg(feature = "stateless_plan")] + let metrics = state.metrics.clone_inner(); + + #[cfg(not(feature = "stateless_plan"))] let metrics = exec.metrics().unwrap(); + // Just verify the metrics can be retrieved (spilling may or may not occur) let spill_count = metrics.spill_count().unwrap_or(0); assert!(spill_count > 0); diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 73ba889c9e40b..bedb7d8c5ee51 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -57,8 +57,10 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use crate::metrics::BaselineMetrics; use crate::sorts::sort::sort_batch; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, @@ -69,6 +71,8 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_common::utils::evaluate_partition_ranges; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_execution::{RecordBatchStream, TaskContext}; use datafusion_physical_expr::LexOrdering; @@ -85,8 +89,6 @@ pub struct PartialSortExec { /// Length of continuous matching columns of input that satisfy /// the required ordering for the sort common_prefix_length: usize, - /// Containing all metrics set created during sort - metrics_set: ExecutionPlanMetricsSet, /// Preserve partitions of input plan. If false, the input partitions /// will be sorted and merged into a single output partition. preserve_partitioning: bool, @@ -94,6 +96,9 @@ pub struct PartialSortExec { fetch: Option, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// Containing all metrics set created during sort + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } impl PartialSortExec { @@ -111,10 +116,11 @@ impl PartialSortExec { input, expr, common_prefix_length, - metrics_set: ExecutionPlanMetricsSet::new(), preserve_partitioning, fetch: None, cache, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), } } @@ -298,7 +304,13 @@ impl ExecutionPlan for PartialSortExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + trace!( "Start PartialSortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, @@ -306,7 +318,8 @@ impl ExecutionPlan for PartialSortExec { context.task_id() ); - let input = self.input.execute(partition, Arc::clone(&context))?; + let input = + execute_input!(0, self.input, partition, Arc::clone(&context), state)?; trace!("End PartialSortExec's input.execute for partition: {partition}"); @@ -321,12 +334,13 @@ impl ExecutionPlan for PartialSortExec { in_mem_batch: RecordBatch::new_empty(Arc::clone(&self.schema())), fetch: self.fetch, is_closed: false, - baseline_metrics: BaselineMetrics::new(&self.metrics_set, partition), + baseline_metrics: BaselineMetrics::new(plan_metrics!(self, state), partition), })) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { - Some(self.metrics_set.clone_inner()) + Some(self.metrics.clone_inner()) } fn statistics(&self) -> Result { @@ -497,6 +511,7 @@ mod tests { use itertools::Itertools; use crate::collect; + use crate::execution_plan::collect_and_get_metrics_of; use crate::expressions::PhysicalSortExpr; use crate::expressions::col; use crate::sorts::sort::SortExec; @@ -1013,7 +1028,7 @@ mod tests { ], )?; - let partial_sort_exec = Arc::new(PartialSortExec::new( + let partial_sort_exec: Arc = Arc::new(PartialSortExec::new( [ PhysicalSortExpr { expr: col("a", &schema)?, @@ -1046,8 +1061,9 @@ mod tests { *partial_sort_exec.schema().field(2).data_type() ); - let result: Vec = collect( - Arc::clone(&partial_sort_exec) as Arc, + let (result, metrics) = collect_and_get_metrics_of( + Arc::clone(&partial_sort_exec), + &partial_sort_exec, task_ctx, ) .await?; @@ -1066,7 +1082,7 @@ mod tests { +-----+------+-------+ "); assert_eq!(result.len(), 2); - let metrics = partial_sort_exec.metrics().unwrap(); + let metrics = metrics.unwrap(); assert!(metrics.elapsed_compute().unwrap() > 0); assert_eq!(metrics.output_rows().unwrap(), 8); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 18cdcbe9debcc..6b10299999bc4 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -27,6 +27,9 @@ use std::sync::Arc; use parking_lot::RwLock; use crate::common::spawn_buffered; +use crate::dynamic_filter::{ + PlannedDynamicFilterPhysicalExpr, make_planned_dynamic_filter, +}; use crate::execution_plan::{Boundedness, CardinalityEffect, EmissionType}; use crate::expressions::PhysicalSortExpr; use crate::filter_pushdown::{ @@ -34,14 +37,15 @@ use crate::filter_pushdown::{ }; use crate::limit::LimitStream; use crate::metrics::{ - BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet, RecordOutput, SpillMetrics, - SplitMetrics, + BaselineMetrics, ExecutionPlanMetricsSet, RecordOutput, SpillMetrics, SplitMetrics, }; use crate::projection::{ProjectionExec, make_with_child, update_ordering}; use crate::sorts::streaming_merge::{SortedSpillFile, StreamingMergeBuilder}; use crate::spill::get_record_batch_memory_size; use crate::spill::in_progress_spill_file::InProgressSpillFile; use crate::spill::spill_manager::{GetSlicedSize, SpillManager}; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::stream::BatchSplitStream; use crate::stream::RecordBatchStreamAdapter; use crate::topk::TopK; @@ -62,10 +66,12 @@ use datafusion_common::{ }; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::MetricsSet; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_physical_expr::LexOrdering; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr::expressions::{DynamicFilterPhysicalExpr, lit}; +use datafusion_physical_expr::expressions::lit; use futures::{StreamExt, TryStreamExt}; use log::{debug, trace}; @@ -881,8 +887,6 @@ pub struct SortExec { pub(crate) input: Arc, /// Sort expressions expr: LexOrdering, - /// Containing all metrics set created during sort - metrics_set: ExecutionPlanMetricsSet, /// Preserve partitions of input plan. If false, the input partitions /// will be sorted and merged into a single output partition. preserve_partitioning: bool, @@ -895,9 +899,42 @@ pub struct SortExec { /// Filter matching the state of the sort for dynamic filter pushdown. /// If `fetch` is `Some`, this will also be set and a TopK operator may be used. /// If `fetch` is `None`, this will be `None`. - filter: Option>>, + filter: Option>, + /// Containing all metrics set created during sort + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, +} + +#[cfg(feature = "stateless_plan")] +mod exec_state { + use crate::dynamic_filter::DynamicFilterPhysicalExpr; + use crate::state::PlanState; + + use super::*; + + pub struct SortExecState { + /// Originated from [`SortExec::filter`]. + pub filter: Option>>, + } + + impl PlanState for SortExecState { + fn as_any(&self) -> &dyn Any { + self + } + + fn dynamic_filters(&self) -> Vec> { + if let Some(filter) = self.filter.as_ref() { + vec![filter.read().expr()] + } else { + vec![] + } + } + } } +#[cfg(feature = "stateless_plan")] +use exec_state::SortExecState; + impl SortExec { /// Create a new sort execution plan that produces a single, /// sorted output partition. @@ -909,12 +946,13 @@ impl SortExec { Self { expr, input, - metrics_set: ExecutionPlanMetricsSet::new(), preserve_partitioning, fetch: None, common_sort_prefix: sort_prefix, cache, filter: None, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), } } @@ -941,28 +979,38 @@ impl SortExec { self } - /// Add or reset `self.filter` to a new `TopKDynamicFilters`. - fn create_filter(&self) -> Arc> { + /// Add or reset `self.filter`. + fn create_filter(&self) -> Arc { let children = self .expr .iter() .map(|sort_expr| Arc::clone(&sort_expr.expr)) .collect::>(); - Arc::new(RwLock::new(TopKDynamicFilters::new(Arc::new( - DynamicFilterPhysicalExpr::new(children, lit(true)), - )))) + Arc::new(make_planned_dynamic_filter(lit(true), children)) + } + + fn execute_filter(&self) -> Option>> { + self.filter.as_ref().map(|filter| { + Arc::new(RwLock::new(TopKDynamicFilters::new( + #[cfg(feature = "stateless_plan")] + Arc::new(filter.to_executable()), + #[cfg(not(feature = "stateless_plan"))] + Arc::clone(filter), + ))) + }) } fn cloned(&self) -> Self { SortExec { input: Arc::clone(&self.input), expr: self.expr.clone(), - metrics_set: self.metrics_set.clone(), preserve_partitioning: self.preserve_partitioning, common_sort_prefix: self.common_sort_prefix.clone(), fetch: self.fetch, cache: self.cache.clone(), filter: self.filter.clone(), + #[cfg(not(feature = "stateless_plan"))] + metrics: self.metrics.clone(), } } @@ -1092,12 +1140,19 @@ impl DisplayAs for SortExec { "SortExec: TopK(fetch={fetch}), expr=[{}], preserve_partitioning=[{preserve_partitioning}]", self.expr )?; + + // Note: in the case of stateless plan stored planned dynamic + // filter is always trivial prior to execution so it doesn't make + // sense to display it. + // TODO: introduce `fmt_as_with_state` to handle such cases. + #[cfg(not(feature = "stateless_plan"))] if let Some(filter) = &self.filter - && let Ok(current) = filter.read().expr().current() + && let Ok(current) = filter.current() && !current.eq(&lit(true)) { write!(f, ", filter=[{current}]")?; } + if !self.common_sort_prefix.is_empty() { write!(f, ", sort_prefix=[")?; let mut first = true; @@ -1190,6 +1245,7 @@ impl ExecutionPlan for SortExec { Ok(Arc::new(new_sort)) } + #[cfg(not(feature = "stateless_plan"))] fn reset_state(self: Arc) -> Result> { let children = self.children().into_iter().cloned().collect(); let new_sort = self.with_new_children(children)?; @@ -1200,7 +1256,7 @@ impl ExecutionPlan for SortExec { .clone(); // Our dynamic filter and execution metrics are the state we need to reset. new_sort.filter = Some(new_sort.create_filter()); - new_sort.metrics_set = ExecutionPlanMetricsSet::new(); + new_sort.metrics = ExecutionPlanMetricsSet::new(); Ok(Arc::new(new_sort)) } @@ -1209,7 +1265,13 @@ impl ExecutionPlan for SortExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + trace!( "Start SortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, @@ -1217,7 +1279,21 @@ impl ExecutionPlan for SortExec { context.task_id() ); - let mut input = self.input.execute(partition, Arc::clone(&context))?; + // Initialize state prior to execute input to possibly provide + // an access to dynamic filters for children nodes. + #[cfg(feature = "stateless_plan")] + let filter = { + let exec_state = state.get_or_init_state(|| SortExecState { + filter: self.execute_filter(), + }); + exec_state.filter.clone() + }; + + #[cfg(not(feature = "stateless_plan"))] + let filter = self.execute_filter(); + + let mut input = + execute_input!(0, self.input, partition, Arc::clone(&context), state)?; let execution_options = &context.session_config().options().execution; @@ -1233,11 +1309,10 @@ impl ExecutionPlan for SortExec { input, 0, Some(*fetch), - BaselineMetrics::new(&self.metrics_set, partition), + BaselineMetrics::new(plan_metrics!(self, state), partition), ))), (true, None) => Ok(input), (false, Some(fetch)) => { - let filter = self.filter.clone(); let mut topk = TopK::try_new( partition, input.schema(), @@ -1246,7 +1321,7 @@ impl ExecutionPlan for SortExec { *fetch, context.session_config().batch_size(), context.runtime_env(), - &self.metrics_set, + plan_metrics!(self, state), Arc::clone(&unwrap_or_internal_err!(filter)), )?; Ok(Box::pin(RecordBatchStreamAdapter::new( @@ -1273,7 +1348,7 @@ impl ExecutionPlan for SortExec { execution_options.sort_spill_reservation_bytes, execution_options.sort_in_place_threshold_bytes, context.session_config().spill_compression(), - &self.metrics_set, + plan_metrics!(self, state), context.runtime_env(), )?; Ok(Box::pin(RecordBatchStreamAdapter::new( @@ -1291,8 +1366,9 @@ impl ExecutionPlan for SortExec { } } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { - Some(self.metrics_set.clone_inner()) + Some(self.metrics.clone_inner()) } fn statistics(&self) -> Result { @@ -1367,7 +1443,7 @@ impl ExecutionPlan for SortExec { if let Some(filter) = &self.filter && config.optimizer.enable_topk_dynamic_filter_pushdown { - child = child.with_self_filter(filter.read().expr()); + child = child.with_self_filter(Arc::clone(filter) as Arc<_>); } Ok(FilterDescription::new().with_child(child)) @@ -1383,7 +1459,7 @@ mod tests { use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::collect; - use crate::execution_plan::Boundedness; + use crate::execution_plan::{Boundedness, collect_and_get_metrics_of, execute_plan}; use crate::expressions::col; use crate::test; use crate::test::TestMemoryExec; @@ -1398,6 +1474,7 @@ mod tests { use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_execution::RecordBatchStream; use datafusion_execution::config::SessionConfig; + use datafusion_execution::metrics::MetricsSet; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr::expressions::{Column, Literal}; @@ -1468,6 +1545,7 @@ mod tests { &self, _partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { Ok(Box::pin(SortedUnboundedStream { schema: Arc::new(self.schema.clone()), @@ -1573,7 +1651,7 @@ mod tests { let input = test::scan_partitioned(partitions); let schema = input.schema(); - let sort_exec = Arc::new(SortExec::new( + let sort_exec: Arc = Arc::new(SortExec::new( [PhysicalSortExpr { expr: col("i", &schema)?, options: SortOptions::default(), @@ -1582,8 +1660,9 @@ mod tests { Arc::new(CoalescePartitionsExec::new(input)), )); - let result = collect( - Arc::clone(&sort_exec) as Arc, + let (result, metrics) = collect_and_get_metrics_of( + Arc::clone(&sort_exec), + &sort_exec, Arc::clone(&task_ctx), ) .await?; @@ -1591,7 +1670,7 @@ mod tests { assert_eq!(result.len(), 2); // Now, validate metrics - let metrics = sort_exec.metrics().unwrap(); + let metrics = metrics.unwrap(); assert_eq!(metrics.output_rows().unwrap(), 10000); assert!(metrics.elapsed_compute().unwrap() > 0); @@ -1644,7 +1723,7 @@ mod tests { // Read the first record batch to assert that our memory limit and sort_spill_reservation // settings trigger the test scenario. { - let mut stream = plan.execute(0, Arc::clone(&task_ctx))?; + let mut stream = execute_plan(Arc::clone(&plan), 0, Arc::clone(&task_ctx))?; let first_batch = stream.next().await.unwrap()?; let batch_reservation = get_reserved_byte_for_record_batch(&first_batch); @@ -1695,7 +1774,7 @@ mod tests { let input = test::scan_partitioned_utf8(200); let schema = input.schema(); - let sort_exec = Arc::new(SortExec::new( + let sort_exec: Arc = Arc::new(SortExec::new( [PhysicalSortExpr { expr: col("i", &schema)?, options: SortOptions::default(), @@ -1704,13 +1783,18 @@ mod tests { Arc::new(CoalescePartitionsExec::new(input)), )); - let result = collect(Arc::clone(&sort_exec) as _, Arc::clone(&task_ctx)).await?; + let (result, metrics) = collect_and_get_metrics_of( + Arc::clone(&sort_exec), + &sort_exec, + Arc::clone(&task_ctx), + ) + .await?; let num_rows = result.iter().map(|batch| batch.num_rows()).sum::(); assert_eq!(num_rows, 20000); // Now, validate metrics - let metrics = sort_exec.metrics().unwrap(); + let metrics = metrics.unwrap(); assert_eq!(metrics.output_rows().unwrap(), 20000); assert!(metrics.elapsed_compute().unwrap() > 0); @@ -1791,7 +1875,7 @@ mod tests { let csv = test::scan_partitioned(partitions); let schema = csv.schema(); - let sort_exec = Arc::new( + let sort_exec: Arc = Arc::new( SortExec::new( [PhysicalSortExpr { expr: col("i", &schema)?, @@ -1803,11 +1887,15 @@ mod tests { .with_fetch(fetch), ); - let result = - collect(Arc::clone(&sort_exec) as _, Arc::clone(&task_ctx)).await?; + let (result, metrics) = collect_and_get_metrics_of( + Arc::clone(&sort_exec), + &sort_exec, + Arc::clone(&task_ctx), + ) + .await?; assert_eq!(result.len(), 1); - let metrics = sort_exec.metrics().unwrap(); + let metrics = metrics.unwrap(); let did_it_spill = metrics.spill_count().unwrap_or(0) > 0; assert_eq!(did_it_spill, expect_spillage, "with fetch: {fetch:?}"); } @@ -1890,7 +1978,7 @@ mod tests { ], )?; - let sort_exec = Arc::new(SortExec::new( + let sort_exec: Arc = Arc::new(SortExec::new( [ PhysicalSortExpr { expr: col("a", &schema)?, @@ -1917,9 +2005,13 @@ mod tests { *sort_exec.schema().field(1).data_type() ); - let result: Vec = - collect(Arc::clone(&sort_exec) as Arc, task_ctx).await?; - let metrics = sort_exec.metrics().unwrap(); + let (result, metrics) = collect_and_get_metrics_of( + Arc::clone(&sort_exec), + &sort_exec, + Arc::clone(&task_ctx), + ) + .await?; + let metrics = metrics.unwrap(); assert!(metrics.elapsed_compute().unwrap() > 0); assert_eq!(metrics.output_rows().unwrap(), 4); assert_eq!(result.len(), 1); @@ -1977,7 +2069,7 @@ mod tests { ], )?; - let sort_exec = Arc::new(SortExec::new( + let sort_exec: Arc = Arc::new(SortExec::new( [ PhysicalSortExpr { expr: col("a", &schema)?, @@ -2001,9 +2093,13 @@ mod tests { assert_eq!(DataType::Float32, *sort_exec.schema().field(0).data_type()); assert_eq!(DataType::Float64, *sort_exec.schema().field(1).data_type()); - let result: Vec = - collect(Arc::clone(&sort_exec) as Arc, task_ctx).await?; - let metrics = sort_exec.metrics().unwrap(); + let (result, metrics) = collect_and_get_metrics_of( + Arc::clone(&sort_exec), + &sort_exec, + Arc::clone(&task_ctx), + ) + .await?; + let metrics = metrics.unwrap(); assert!(metrics.elapsed_compute().unwrap() > 0); assert_eq!(metrics.output_rows().unwrap(), 8); assert_eq!(result.len(), 1); @@ -2384,10 +2480,14 @@ mod tests { TestMemoryExec::try_new_exec(std::slice::from_ref(&batches), schema, None)?, )); - let sorted_batches = - collect(Arc::clone(&sort_exec), Arc::clone(&task_ctx)).await?; + let (sorted_batches, metrics) = collect_and_get_metrics_of( + Arc::clone(&sort_exec), + &sort_exec, + Arc::clone(&task_ctx), + ) + .await?; - let metrics = sort_exec.metrics().expect("sort have metrics"); + let metrics = metrics.expect("sort have metrics"); // assert output { diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 4b26f84099505..4bfb5d07e6eae 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -22,9 +22,11 @@ use std::sync::Arc; use crate::common::spawn_buffered; use crate::limit::LimitStream; -use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use crate::metrics::BaselineMetrics; use crate::projection::{ProjectionExec, make_with_child, update_ordering}; use crate::sorts::streaming_merge::StreamingMergeBuilder; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, @@ -33,6 +35,8 @@ use crate::{ use datafusion_common::{Result, assert_eq_or_internal_err, internal_err}; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::MemoryConsumer; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, OrderingRequirements}; use crate::execution_plan::{EvaluationType, SchedulingType}; @@ -88,8 +92,6 @@ pub struct SortPreservingMergeExec { input: Arc, /// Sort expressions expr: LexOrdering, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// Optional number of rows to fetch. Stops producing rows after this fetch fetch: Option, /// Cache holding plan properties like equivalences, output partitioning etc. @@ -98,6 +100,9 @@ pub struct SortPreservingMergeExec { /// /// See [`Self::with_round_robin_repartition`] for more information. enable_round_robin_repartition: bool, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } impl SortPreservingMergeExec { @@ -107,10 +112,11 @@ impl SortPreservingMergeExec { Self { input, expr, - metrics: ExecutionPlanMetricsSet::new(), fetch: None, cache, enable_round_robin_repartition: true, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), } } @@ -238,6 +244,7 @@ impl ExecutionPlan for SortPreservingMergeExec { Some(Arc::new(Self { input: Arc::clone(&self.input), expr: self.expr.clone(), + #[cfg(not(feature = "stateless_plan"))] metrics: self.metrics.clone(), fetch: limit, cache: self.cache.clone(), @@ -279,7 +286,13 @@ impl ExecutionPlan for SortPreservingMergeExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + trace!("Start SortPreservingMergeExec::execute for partition: {partition}"); assert_eq_or_internal_err!( partition, @@ -303,7 +316,7 @@ impl ExecutionPlan for SortPreservingMergeExec { ), 1 => match self.fetch { Some(fetch) => { - let stream = self.input.execute(0, context)?; + let stream = execute_input!(0, self.input, 0, context, state)?; debug!( "Done getting stream for SortPreservingMergeExec::execute with 1 input with {fetch}" ); @@ -311,11 +324,11 @@ impl ExecutionPlan for SortPreservingMergeExec { stream, 0, Some(fetch), - BaselineMetrics::new(&self.metrics, partition), + BaselineMetrics::new(plan_metrics!(self, state), partition), ))) } None => { - let stream = self.input.execute(0, context); + let stream = execute_input!(0, self.input, 0, context, state); debug!( "Done getting stream for SortPreservingMergeExec::execute with 1 input without fetch" ); @@ -325,8 +338,13 @@ impl ExecutionPlan for SortPreservingMergeExec { _ => { let receivers = (0..input_partitions) .map(|partition| { - let stream = - self.input.execute(partition, Arc::clone(&context))?; + let stream = execute_input!( + 0, + self.input, + partition, + Arc::clone(&context), + state, + )?; Ok(spawn_buffered(stream, 1)) }) .collect::>()?; @@ -339,7 +357,10 @@ impl ExecutionPlan for SortPreservingMergeExec { .with_streams(receivers) .with_schema(schema) .with_expressions(&self.expr) - .with_metrics(BaselineMetrics::new(&self.metrics, partition)) + .with_metrics(BaselineMetrics::new( + plan_metrics!(self, state), + partition, + )) .with_batch_size(context.session_config().batch_size()) .with_fetch(self.fetch) .with_reservation(reservation) @@ -355,6 +376,7 @@ impl ExecutionPlan for SortPreservingMergeExec { } } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -410,7 +432,9 @@ mod tests { use super::*; use crate::coalesce_batches::CoalesceBatchesExec; use crate::coalesce_partitions::CoalescePartitionsExec; - use crate::execution_plan::{Boundedness, EmissionType}; + use crate::execution_plan::{ + Boundedness, EmissionType, collect_and_get_metrics_of, execute_plan, + }; use crate::expressions::col; use crate::metrics::{MetricValue, Timestamp}; use crate::repartition::RepartitionExec; @@ -432,6 +456,7 @@ mod tests { use datafusion_common_runtime::SpawnedTask; use datafusion_execution::RecordBatchStream; use datafusion_execution::config::SessionConfig; + use datafusion_execution::metrics::ExecutionPlanMetricsSet; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr::expressions::Column; @@ -1091,7 +1116,9 @@ mod tests { let sender = builder.tx(); - let mut stream = batches.execute(partition, Arc::clone(&task_ctx)).unwrap(); + let mut stream = + execute_plan(Arc::clone(&batches), partition, Arc::clone(&task_ctx)) + .unwrap(); builder.spawn(async move { while let Some(batch) = stream.next().await { sender.send(batch).await.unwrap(); @@ -1160,11 +1187,13 @@ mod tests { .into(); let exec = TestMemoryExec::try_new_exec(&[vec![b1], vec![b2]], schema, None).unwrap(); - let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); + let merge: Arc = + Arc::new(SortPreservingMergeExec::new(sort, exec)); - let collected = collect(Arc::clone(&merge) as Arc, task_ctx) - .await - .unwrap(); + let (collected, metrics) = + collect_and_get_metrics_of(Arc::clone(&merge), &merge, task_ctx) + .await + .unwrap(); assert_snapshot!(batches_to_string(collected.as_slice()), @r" +----+---+ | a | b | @@ -1177,7 +1206,7 @@ mod tests { "); // Now, validate metrics - let metrics = merge.metrics().unwrap(); + let metrics = metrics.unwrap(); assert_eq!(metrics.output_rows().unwrap(), 4); assert!(metrics.elapsed_compute().unwrap() > 0); @@ -1402,6 +1431,7 @@ mod tests { &self, partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { Ok(Box::pin(CongestedStream { schema: Arc::new(self.schema.clone()), diff --git a/datafusion/physical-plan/src/state.rs b/datafusion/physical-plan/src/state.rs new file mode 100644 index 0000000000000..869e39e5ca5db --- /dev/null +++ b/datafusion/physical-plan/src/state.rs @@ -0,0 +1,431 @@ +// 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, + sync::{Arc, OnceLock, Weak}, +}; + +use arrow::array::RecordBatch; +use datafusion_common::{ + Result, internal_err, + tree_node::{Transformed, TreeNode}, +}; +use datafusion_execution::{ + RecordBatchStream, SendableRecordBatchStream, metrics::MetricsSet, +}; +use datafusion_physical_expr::{ + PhysicalExpr, + expressions::{DynamicFilterPhysicalExpr, PlannedDynamicFilterPhysicalExpr}, +}; +use futures::{Stream, StreamExt}; +use parking_lot::Mutex; + +use crate::{ExecutionPlan, WorkTable, metrics::ExecutionPlanMetricsSet}; + +/// [`PlanStateNode`] contains the state required during plan execution. +/// It is maintained so that each partition of a given plan receives the +/// same state node during a single query execution. +/// +/// A [`PlanStateNode`] tree, built during the [`ExecutionPlan::execute`] stage, +/// mirrors the structure of the [`ExecutionPlan`] tree itself. It is designed +/// to store plan state that is created during execution and must later be +/// associated with the corresponding plan nodes—for example, metrics. +/// +/// Each plan can store its state in the corresponding state node by implementing +/// [`PlanState`] for its specific state structure. +/// +/// # Passing data to child nodes +/// +/// State tree allows to pass state from some plan to children, for example, +/// work tables for recursive queries or dynamic filters. For the details, +/// see [`PlanState`] trait. +/// +/// [`ExecutionPlan`]: crate::ExecutionPlan +/// [`ExecutionPlan::execute`]: crate::ExecutionPlan::execute +/// +pub struct PlanStateNode { + /// Corresponding plan node. + pub plan_node: Arc, + /// Parent of the current node if exists. + /// [`None`] if node is root. + parent_node: Option>, + /// Plan specific state initialized once per execution. + pub(super) plan_state: OnceLock>, + /// Metrics associated with this plan node. + pub metrics: ExecutionPlanMetricsSet, + /// State for each plan child lazily initialized. + children_state: Mutex>]>>, +} + +impl PlanStateNode { + /// Make a new [`PlanStateNode`]. + fn new( + plan_node: Arc, + parent_node: Option>, + ) -> Self { + let num_children = plan_node.children().len(); + Self { + plan_node, + parent_node, + plan_state: OnceLock::default(), + metrics: ExecutionPlanMetricsSet::new(), + children_state: Mutex::new( + (0..num_children).map(|_| OnceLock::default()).collect(), + ), + } + } + + /// Make a new arced [`PlanStateNode`]. + fn new_arc( + plan_node: Arc, + parent_node: Option>, + ) -> Arc { + Arc::new(Self::new(plan_node, parent_node)) + } + + /// Make a new [`PlanStateNode`] for a root plan node. + pub fn new_root(plan_node: Arc) -> Self { + Self::new(plan_node, None) + } + + /// Make a new arced [`PlanStateNode`] for a root plan node. + pub fn new_root_arc(plan_node: Arc) -> Arc { + Arc::new(Self::new_root(plan_node)) + } + + /// Find metrics of the plan within state tree. Returns [`None`] if the + /// `plan` is not found among state tree plan nodes. + pub fn metrics_of( + self: &Arc, + plan: &Arc, + ) -> Option { + let mut metrics = None; + accept_state(self, &mut |state: &Arc| -> Result { + if Arc::ptr_eq(&state.plan_node, plan) { + metrics = Some(state.metrics.clone_inner()); + Ok(false) + } else { + Ok(true) + } + }) + .unwrap(); + metrics + } + + /// Get or init state using passed `f`. Returns a reference to the state. + /// + /// # Panics + /// + /// State is already initialized and cannot be downcast to `S`. + /// + pub fn get_or_init_state(&self, f: impl FnOnce() -> S) -> &S { + self.plan_state + .get_or_init(|| Arc::new(f())) + .as_any() + .downcast_ref::() + .unwrap() + } + + /// Make a child state node if it not initialized and return it. + /// + /// # Panics + /// + /// `child_idx` is more than the corresponding plan children number. + /// + pub fn child_state(self: &Arc, child_idx: usize) -> Arc { + Arc::clone(self.children_state.lock()[child_idx].get_or_init(|| { + Self::new_arc( + Arc::clone(self.plan_node.children()[child_idx]), + // Set node parent. + Some(Arc::downgrade(self)), + ) + })) + } + + /// Lookup for the last [`WorkTable`] owner node over the path from root + /// to the current node. + /// + /// This function is intended to be called by plan node that should operate + /// with a work table during [`ExecutionPlan::execute`] call to find table set + /// by work table owner, typically it is a [`RecursiveQueryExec`]. + /// + /// [`RecursiveQueryExec`]: crate::recursive_query::RecursiveQueryExec + /// + pub fn work_table(&self) -> Option> { + self.inspect_root_path(|node| { + node.plan_state.get().and_then(|state| state.work_table()) + }) + } + + /// Replace all planned dynamic filters in the given expression, + /// converting them into executable versions by deriving shared + /// state from the filter owner. + /// + /// This function is intended to be called by a plan node that supports + /// dynamic filters during [`ExecutionPlan::execute`]. It converts stored + /// planning-time filters into execution-time filters by looking up the + /// filters stored in one of the parent nodes along the path to the + /// state tree root. + /// + pub fn planned_dynamic_filter_to_executable( + &self, + expr: Arc, + ) -> Result> { + expr.transform_up(|expr| { + let Some(dynamic_filter) = expr + .as_any() + .downcast_ref::() + else { + return Ok(Transformed::no(expr)); + }; + let filter = match self.dynamic_filter_for(dynamic_filter) { + None => { + return internal_err!( + "dynamic filter cannot be resolved to executable" + ); + } + Some(filter) => filter, + }; + Ok(Transformed::yes(filter as _)) + }) + .map(|tnr| tnr.data) + } + + /// Lookup for the execution time dynamic filter by its origin. + fn dynamic_filter_for( + &self, + origin: &PlannedDynamicFilterPhysicalExpr, + ) -> Option> { + self.inspect_root_path(|node| { + if let Some(state) = node.plan_state.get() { + for filter in state.dynamic_filters() { + if let Some(res) = filter.as_dynamic_for(origin) { + return Some(res); + } + } + } + None + }) + } + + fn inspect_root_path(&self, f: impl Fn(&PlanStateNode) -> Option) -> Option { + if let Some(res) = f(self) { + return Some(res); + } + let mut current_node = self.parent_node.as_ref().and_then(|p| p.upgrade()); + while let Some(node) = current_node { + if let Some(res) = f(&node) { + return Some(res); + } + current_node = node.parent_node.as_ref().and_then(|p| p.upgrade()); + } + None + } +} + +/// Generic execution stage plan state. +pub trait PlanState: Send + Sync + 'static { + /// Returns the state as [`Any`] so that it can be downcast to + /// a specific implementation. + fn as_any(&self) -> &dyn Any; + + /// Return dynamic filters maintained by this node. + /// + /// This function is used to push shared mutable dynamic filters + /// from an owner to a child that accepted the filter during the + /// planning stage via filter push-down optimization. + /// + fn dynamic_filters(&self) -> Vec> { + vec![] + } + + /// Return [`WorkTable`] maintained by this node. + fn work_table(&self) -> Option> { + None + } +} + +/// Describes a data associated with a [`PlanStateNode`]. +pub struct WithPlanStateNode { + inner: T, + state: Arc, +} + +impl WithPlanStateNode { + /// Make a new [`WithPlanStateNode`]. + pub fn new(inner: T, state: Arc) -> Self { + Self { inner, state } + } + + /// Project an inner data. + pub fn as_inner(&self) -> &T { + &self.inner + } + + /// Project an inner mutable data. + pub fn as_inner_mut(&mut self) -> &mut T { + &mut self.inner + } + + /// Project a state. + pub fn state(&self) -> &Arc { + &self.state + } + + /// Borrow an inner data, preserving the state node. + pub fn as_ref(&self) -> WithPlanStateNode<&T> { + WithPlanStateNode { + inner: &self.inner, + state: Arc::clone(&self.state), + } + } + + /// Map an inner data, preserving the state node. + pub fn map(self, f: impl FnOnce(T) -> V) -> WithPlanStateNode { + WithPlanStateNode { + inner: f(self.inner), + state: self.state, + } + } + + /// Try to map an inner data, preserving the state node. + pub fn try_map( + self, + f: impl FnOnce(T) -> Result, + ) -> Result> { + let inner = f(self.inner)?; + Ok(WithPlanStateNode { + inner, + state: self.state, + }) + } + + /// Try to apply async map `f`, preserving the state node. + pub async fn try_map_async( + self, + f: impl FnOnce(T) -> Fut, + ) -> Result> + where + Fut: Future>, + { + let inner = f(self.inner).await?; + Ok(WithPlanStateNode { + inner, + state: self.state, + }) + } + + /// Consume `self` and convert into inner. + pub fn into_inner(self) -> T { + self.inner + } + + /// Consume `self` and convert into node state. + pub fn into_state(self) -> Arc { + self.state + } + + /// Return stored parts. + pub fn into_parts(self) -> (T, Arc) { + (self.inner, self.state) + } +} + +impl WithPlanStateNode> { + /// Represent self as a result. + pub fn as_result(self) -> Result> { + self.inner.map(|inner| WithPlanStateNode { + inner, + state: self.state, + }) + } +} + +impl Stream for WithPlanStateNode { + type Item = Result; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + self.as_inner_mut().poll_next_unpin(cx) + } + + fn size_hint(&self) -> (usize, Option) { + self.as_inner().size_hint() + } +} + +impl RecordBatchStream for WithPlanStateNode { + fn schema(&self) -> arrow_schema::SchemaRef { + self.as_inner().schema() + } +} + +/// Visit all children of this state using passing `visitor`. +pub fn accept_state( + state: &Arc, + visitor: &mut V, +) -> Result { + if !visitor.pre_visit(state)? { + return Ok(false); + }; + for i in 0..state.plan_node.children().len() { + if !accept_state(&state.child_state(i), visitor)? { + return Ok(false); + } + } + if !visitor.post_visit(state)? { + return Ok(false); + }; + Ok(true) +} + +/// Trait that implements the [Visitor +/// pattern](https://en.wikipedia.org/wiki/Visitor_pattern) for a +/// depth first walk of [`PlanStateNode`] nodes. `pre_visit` is called +/// before any children are visited, and then `post_visit` is called +/// after all children have been visited. +pub trait ExecutionPlanStateVisitor { + /// The type of error returned by this visitor. + type Error; + + /// Invoked on an [`PlanStateNode`] before any of its child have + /// been visited. If Ok(true) is returned, the recursion continues. + /// If Err(..) or Ok(false) are returned, the recursion stops immediately + /// and the error, if any, is returned. + fn pre_visit(&mut self, state: &Arc) -> Result; + + /// Invoked on an [`PlanStateNode`] plan *after* all of its child + /// inputs have been visited. The return value is handled the same + /// as the return value of `pre_visit`. + fn post_visit(&mut self, _state: &Arc) -> Result { + Ok(true) + } +} + +impl ExecutionPlanStateVisitor for F +where + F: FnMut(&Arc) -> Result, +{ + type Error = E; + + fn pre_visit(&mut self, state: &Arc) -> Result { + (self)(state) + } +} diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index 8b2ea1006893e..00a729a58a0a2 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -27,6 +27,8 @@ use super::metrics::ExecutionPlanMetricsSet; use super::metrics::{BaselineMetrics, SplitMetrics}; use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream}; use crate::displayable; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use datafusion_common::{Result, exec_err}; @@ -326,11 +328,19 @@ impl RecordBatchReceiverStreamBuilder { input: Arc, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) { let output = self.tx(); + #[cfg(feature = "stateless_plan")] + let state = Arc::clone(state); self.inner.spawn(async move { - let mut stream = match input.execute(partition, context) { + #[cfg(feature = "stateless_plan")] + let stream = input.execute(partition, context, &state); + #[cfg(not(feature = "stateless_plan"))] + let stream = input.execute(partition, context); + + let mut stream = match stream { Err(e) => { // If send fails, the plan being torn down, there // is no place to send the error and no reason to continue. @@ -743,6 +753,36 @@ mod test { consume(input, max_batches).await } + #[cfg(feature = "stateless_plan")] + #[must_use] + fn run_input( + builder: &mut RecordBatchReceiverStreamBuilder, + input: Arc, + partition: usize, + task_ctx: Arc, + ) -> Arc { + let state = PlanStateNode::new_root_arc(Arc::clone(&input)); + builder.run_input(input, partition, task_ctx, &state); + state + } + + struct EmptyState {} + + impl Drop for EmptyState { + fn drop(&mut self) {} + } + + #[cfg(not(feature = "stateless_plan"))] + fn run_input( + builder: &mut RecordBatchReceiverStreamBuilder, + input: Arc, + partition: usize, + task_ctx: Arc, + ) -> EmptyState { + builder.run_input(input, partition, task_ctx); + EmptyState {} + } + #[tokio::test] async fn record_batch_receiver_stream_drop_cancel() { let task_ctx = Arc::new(TaskContext::default()); @@ -754,14 +794,16 @@ mod test { // Configure a RecordBatchReceiverStream to consume the input let mut builder = RecordBatchReceiverStream::builder(schema, 2); - builder.run_input(Arc::new(input), 0, Arc::clone(&task_ctx)); + + let state = run_input(&mut builder, Arc::new(input), 0, Arc::clone(&task_ctx)); let stream = builder.build(); // Input should still be present assert!(std::sync::Weak::strong_count(&refs) > 0); - // Drop the stream, ensure the refs go to zero + // Drop the stream and state, ensure the refs go to zero drop(stream); + drop(state); assert_strong_count_converges_to_zero(refs).await; } @@ -781,7 +823,12 @@ mod test { .with_use_task(false); let mut builder = RecordBatchReceiverStream::builder(schema, 2); - builder.run_input(Arc::new(error_stream), 0, Arc::clone(&task_ctx)); + let _state = run_input( + &mut builder, + Arc::new(error_stream), + 0, + Arc::clone(&task_ctx), + ); let mut stream = builder.build(); // Get the first result, which should be an error @@ -844,12 +891,15 @@ mod test { // Configure a RecordBatchReceiverStream to consume all the input partitions let mut builder = RecordBatchReceiverStream::builder(input.schema(), num_partitions); + let mut states = vec![]; for partition in 0..num_partitions { - builder.run_input( + let state = run_input( + &mut builder, Arc::clone(&input) as Arc, partition, Arc::clone(&task_ctx), ); + states.push(state); } let mut stream = builder.build(); diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index c8b8d95718cb8..6bb9c02ad28c5 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -26,16 +26,20 @@ use crate::coop::make_cooperative; use crate::display::{ProjectSchemaDisplay, display_orderings}; use crate::execution_plan::{Boundedness, EmissionType, SchedulingType}; use crate::limit::LimitStream; -use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use crate::metrics::BaselineMetrics; use crate::projection::{ ProjectionExec, all_alias_free_columns, new_projections_for_columns, update_ordering, }; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::stream::RecordBatchStreamAdapter; use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; use arrow::datatypes::{Schema, SchemaRef}; use datafusion_common::{Result, internal_err, plan_err}; use datafusion_execution::TaskContext; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use async_trait::async_trait; @@ -68,6 +72,7 @@ pub struct StreamingTableExec { infinite: bool, limit: Option, cache: PlanProperties, + #[cfg(not(feature = "stateless_plan"))] metrics: ExecutionPlanMetricsSet, } @@ -112,6 +117,7 @@ impl StreamingTableExec { infinite, limit, cache, + #[cfg(not(feature = "stateless_plan"))] metrics: ExecutionPlanMetricsSet::new(), }) } @@ -263,7 +269,13 @@ impl ExecutionPlan for StreamingTableExec { &self, partition: usize, ctx: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::plan_metrics; + let stream = self.partitions[partition].execute(Arc::clone(&ctx)); let projected_stream = match self.projection.clone() { Some(projection) => Box::pin(RecordBatchStreamAdapter::new( @@ -279,7 +291,8 @@ impl ExecutionPlan for StreamingTableExec { Ok(match self.limit { None => stream, Some(fetch) => { - let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + let baseline_metrics = + BaselineMetrics::new(plan_metrics!(self, state), partition); Box::pin(LimitStream::new(stream, 0, Some(fetch), baseline_metrics)) } }) @@ -323,6 +336,7 @@ impl ExecutionPlan for StreamingTableExec { .map(|e| Some(Arc::new(e) as _)) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -336,6 +350,7 @@ impl ExecutionPlan for StreamingTableExec { infinite: self.infinite, limit, cache: self.cache.clone(), + #[cfg(not(feature = "stateless_plan"))] metrics: self.metrics.clone(), })) } diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index c94b5a4131397..9ff8f19ff2158 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -29,7 +29,8 @@ use crate::ExecutionPlan; use crate::common; use crate::execution_plan::{Boundedness, EmissionType}; use crate::memory::MemoryStream; -use crate::metrics::MetricsSet; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::stream::RecordBatchStreamAdapter; use crate::streaming::PartitionStream; use crate::{DisplayAs, DisplayFormatType, PlanProperties}; @@ -39,6 +40,7 @@ use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{ Result, Statistics, assert_or_internal_err, config::ConfigOptions, project_schema, }; +use datafusion_execution::metrics::MetricsSet; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::equivalence::{ OrderingEquivalenceClass, ProjectionMapping, @@ -47,7 +49,7 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, Partitioning}; -use futures::{Future, FutureExt}; +use futures::{Future, FutureExt, StreamExt}; pub mod exec; @@ -146,7 +148,7 @@ impl ExecutionPlan for TestMemoryExec { self: Arc, _: Vec>, ) -> Result> { - unimplemented!() + Ok(self) } fn repartitioned( @@ -161,10 +163,12 @@ impl ExecutionPlan for TestMemoryExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { self.open(partition, context) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { unimplemented!() } @@ -551,3 +555,79 @@ macro_rules! assert_join_metrics { } #[cfg(test)] pub(crate) use assert_join_metrics; + +pub async fn collect_with( + exec: Arc, + task_ctx: Arc, + mut f: impl FnMut(usize, Result) -> Result<()>, +) -> Result { + #[cfg(feature = "stateless_plan")] + let state = PlanStateNode::new_root_arc(Arc::clone(&exec)); + + for i in 0..exec.properties().partitioning.partition_count() { + #[cfg(feature = "stateless_plan")] + let mut stream = exec.execute(i, Arc::clone(&task_ctx), &state)?; + + #[cfg(not(feature = "stateless_plan"))] + let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; + while let Some(result) = stream.next().await { + f(i, result)?; + } + } + + #[cfg(feature = "stateless_plan")] + let metrics = state.metrics.clone_inner(); + + #[cfg(not(feature = "stateless_plan"))] + let metrics = exec.metrics().unwrap(); + + Ok(metrics) +} + +pub async fn collect_partitions( + exec: Arc, + task_ctx: Arc, +) -> Result<(Vec>, MetricsSet)> { + let mut partitions = vec![vec![]; exec.properties().partitioning.partition_count()]; + + let metrics = collect_with(exec, task_ctx, |i, batch| { + partitions[i].push(batch?); + Ok(()) + }) + .await?; + + Ok((partitions, metrics)) +} + +pub async fn collect_batches( + exec: Arc, + task_ctx: Arc, +) -> Result<(Vec, MetricsSet)> { + let mut batches = vec![]; + + let metrics = collect_with(exec, task_ctx, |_i, batch| { + let batch = batch?; + if batch.num_rows() > 0 { + batches.push(batch); + } + Ok(()) + }) + .await?; + + Ok((batches, metrics)) +} + +pub async fn collect_counting_rows( + exec: Arc, + task_ctx: Arc, +) -> Result<(usize, MetricsSet)> { + let mut rows_count = 0; + + let metrics = collect_with(exec, task_ctx, |_i, batch| { + rows_count += batch?.num_rows(); + Ok(()) + }) + .await?; + + Ok((rows_count, metrics)) +} diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index 4507cccba05a9..462cdc9749fb3 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -24,6 +24,8 @@ use std::{ task::{Context, Poll}, }; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, common, @@ -212,6 +214,7 @@ impl ExecutionPlan for MockExec { &self, partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { assert_eq!(partition, 0); @@ -369,14 +372,14 @@ impl ExecutionPlan for BarrierExec { } fn children(&self) -> Vec<&Arc> { - unimplemented!() + vec![] } fn with_new_children( self: Arc, _: Vec>, ) -> Result> { - unimplemented!() + Ok(self) } /// Returns a stream which yields data @@ -384,6 +387,7 @@ impl ExecutionPlan for BarrierExec { &self, partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { assert!(partition < self.data.len()); @@ -492,14 +496,14 @@ impl ExecutionPlan for ErrorExec { } fn children(&self) -> Vec<&Arc> { - unimplemented!() + vec![] } fn with_new_children( self: Arc, _: Vec>, ) -> Result> { - unimplemented!() + Ok(self) } /// Returns a stream which yields data @@ -507,6 +511,7 @@ impl ExecutionPlan for ErrorExec { &self, partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { internal_err!("ErrorExec, unsurprisingly, errored in partition {partition}") } @@ -596,6 +601,7 @@ impl ExecutionPlan for StatisticsExec { &self, _partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { unimplemented!("This plan only serves for testing statistics") } @@ -704,6 +710,7 @@ impl ExecutionPlan for BlockingExec { &self, _partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { Ok(Box::pin(BlockingStream { schema: Arc::clone(&self.schema), @@ -850,6 +857,7 @@ impl ExecutionPlan for PanicExec { &self, partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] _state: &Arc, ) -> Result { Ok(Box::pin(PanicStream { partition, diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index ebac497f4fbc3..8903860aa48f1 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -1067,6 +1067,8 @@ impl RecordBatchStore { #[cfg(test)] mod tests { + use crate::dynamic_filter::make_executable_dynamic_filter; + use super::*; use arrow::array::{Float64Array, Int32Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Schema}; @@ -1149,7 +1151,7 @@ mod tests { runtime, &metrics, Arc::new(RwLock::new(TopKDynamicFilters::new(Arc::new( - DynamicFilterPhysicalExpr::new(vec![], lit(true)), + make_executable_dynamic_filter(lit(true), vec![]), )))), )?; @@ -1222,7 +1224,7 @@ mod tests { let metrics = ExecutionPlanMetricsSet::new(); // Create a dynamic filter that we'll check for completion - let dynamic_filter = Arc::new(DynamicFilterPhysicalExpr::new(vec![], lit(true))); + let dynamic_filter = Arc::new(make_executable_dynamic_filter(lit(true), vec![])); let dynamic_filter_clone = Arc::clone(&dynamic_filter); // Create a TopK instance diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index d27c81b968490..7b7be919c04c8 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -30,7 +30,6 @@ use super::{ ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, - metrics::{ExecutionPlanMetricsSet, MetricsSet}, }; use crate::execution_plan::{ InvariantLevel, boundedness_from_children, check_default_invariants, @@ -39,6 +38,8 @@ use crate::execution_plan::{ use crate::filter_pushdown::{FilterDescription, FilterPushdownPhase}; use crate::metrics::BaselineMetrics; use crate::projection::{ProjectionExec, make_with_child}; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::stream::ObservedStream; use arrow::datatypes::{Field, Schema, SchemaRef}; @@ -49,6 +50,8 @@ use datafusion_common::{ Result, assert_or_internal_err, exec_err, internal_datafusion_err, }; use datafusion_execution::TaskContext; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr, calculate_union}; use futures::Stream; @@ -97,10 +100,11 @@ use tokio::macros::support::thread_rng_n; pub struct UnionExec { /// Input execution plan inputs: Vec>, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } impl UnionExec { @@ -117,8 +121,9 @@ impl UnionExec { let cache = Self::compute_properties(&inputs, schema).unwrap(); UnionExec { inputs, - metrics: ExecutionPlanMetricsSet::new(), cache, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), } } @@ -146,8 +151,9 @@ impl UnionExec { let cache = Self::compute_properties(&inputs, schema).unwrap(); Ok(Arc::new(UnionExec { inputs, - metrics: ExecutionPlanMetricsSet::new(), cache, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), })) } } @@ -266,14 +272,20 @@ impl ExecutionPlan for UnionExec { &self, mut partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; trace!( "Start UnionExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id() ); - let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + let baseline_metrics = + BaselineMetrics::new(plan_metrics!(self, state), partition); // record the tiny amount of work done in this function so // elapsed_compute is reported as non zero let elapsed_compute = baseline_metrics.elapsed_compute().clone(); @@ -283,7 +295,7 @@ impl ExecutionPlan for UnionExec { for input in self.inputs.iter() { // Calculate whether partition belongs to the current partition if partition < input.output_partitioning().partition_count() { - let stream = input.execute(partition, context)?; + let stream = execute_input!(0, input, partition, context, state)?; debug!("Found a Union partition to execute"); return Ok(Box::pin(ObservedStream::new( stream, @@ -300,6 +312,7 @@ impl ExecutionPlan for UnionExec { exec_err!("Partition {partition} not found in Union") } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -408,10 +421,11 @@ impl ExecutionPlan for UnionExec { pub struct InterleaveExec { /// Input execution plan inputs: Vec>, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } impl InterleaveExec { @@ -424,8 +438,9 @@ impl InterleaveExec { let cache = Self::compute_properties(&inputs)?; Ok(InterleaveExec { inputs, - metrics: ExecutionPlanMetricsSet::new(), cache, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), }) } @@ -502,14 +517,20 @@ impl ExecutionPlan for InterleaveExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; trace!( "Start InterleaveExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id() ); - let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + let baseline_metrics = + BaselineMetrics::new(plan_metrics!(self, state), partition); // record the tiny amount of work done in this function so // elapsed_compute is reported as non zero let elapsed_compute = baseline_metrics.elapsed_compute().clone(); @@ -518,7 +539,9 @@ impl ExecutionPlan for InterleaveExec { let mut input_stream_vec = vec![]; for input in self.inputs.iter() { if partition < input.output_partitioning().partition_count() { - input_stream_vec.push(input.execute(partition, Arc::clone(&context))?); + let stream = + execute_input!(0, input, partition, Arc::clone(&context), state)?; + input_stream_vec.push(stream); } else { // Do not find a partition to execute break; @@ -541,6 +564,7 @@ impl ExecutionPlan for InterleaveExec { exec_err!("Partition {partition} not found in InterleaveExec") } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 5fef754e80780..975bb297558ad 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -22,10 +22,11 @@ use std::task::{Poll, ready}; use std::{any::Any, sync::Arc}; use super::metrics::{ - self, BaselineMetrics, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, - RecordOutput, + self, BaselineMetrics, ExecutionPlanMetricsSet, MetricBuilder, RecordOutput, }; use super::{DisplayAs, ExecutionPlanProperties, PlanProperties}; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayFormatType, Distribution, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, @@ -47,6 +48,8 @@ use datafusion_common::{ internal_err, }; use datafusion_execution::TaskContext; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::MetricsSet; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::Column; @@ -71,10 +74,11 @@ pub struct UnnestExec { struct_column_indices: Vec, /// Options options: UnnestOptions, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } impl UnnestExec { @@ -99,8 +103,9 @@ impl UnnestExec { list_column_indices, struct_column_indices, options, - metrics: Default::default(), cache, + #[cfg(not(feature = "stateless_plan"))] + metrics: Default::default(), }) } @@ -250,9 +255,15 @@ impl ExecutionPlan for UnnestExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { - let input = self.input.execute(partition, context)?; - let metrics = UnnestMetrics::new(partition, &self.metrics); + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + + let input = execute_input!(0, self.input, partition, context, state)?; + let metrics = UnnestMetrics::new(partition, plan_metrics!(self, state)); Ok(Box::pin(UnnestStream { input, @@ -264,6 +275,7 @@ impl ExecutionPlan for UnnestExec { })) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 987a400ec369e..c44215ef5bf40 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -28,7 +28,9 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::utils::create_schema; -use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use crate::metrics::BaselineMetrics; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, window_equivalence_properties, @@ -55,6 +57,8 @@ use datafusion_common::{ HashMap, Result, arrow_datafusion_err, exec_datafusion_err, exec_err, }; use datafusion_execution::TaskContext; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_expr::ColumnarValue; use datafusion_expr::window_state::{PartitionBatchState, WindowAggState}; use datafusion_physical_expr::window::{ @@ -81,8 +85,6 @@ pub struct BoundedWindowAggExec { window_expr: Vec>, /// Schema after the window is run schema: SchemaRef, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// Describes how the input is ordered relative to the partition keys pub input_order_mode: InputOrderMode, /// Partition by indices that define ordering @@ -96,6 +98,9 @@ pub struct BoundedWindowAggExec { cache: PlanProperties, /// If `can_rerepartition` is false, partition_keys is always empty. can_repartition: bool, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } impl BoundedWindowAggExec { @@ -131,11 +136,12 @@ impl BoundedWindowAggExec { input, window_expr, schema, - metrics: ExecutionPlanMetricsSet::new(), input_order_mode, ordered_partition_by_indices, cache, can_repartition, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), }) } @@ -351,19 +357,26 @@ impl ExecutionPlan for BoundedWindowAggExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { - let input = self.input.execute(partition, context)?; + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + + let input = execute_input!(0, self.input, partition, context, state)?; let search_mode = self.get_search_algo()?; let stream = Box::pin(BoundedWindowAggStream::new( Arc::clone(&self.schema), self.window_expr.clone(), input, - BaselineMetrics::new(&self.metrics, partition), + BaselineMetrics::new(plan_metrics!(self, state), partition), search_mode, )?); Ok(stream) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -1244,6 +1257,7 @@ mod tests { use std::time::Duration; use crate::common::collect; + use crate::execution_plan::execute_plan; use crate::expressions::PhysicalSortExpr; use crate::projection::{ProjectionExec, ProjectionExpr}; use crate::streaming::{PartitionStream, StreamingTableExec}; @@ -1682,7 +1696,8 @@ mod tests { ) .map(|e| Arc::new(e) as Arc)?; - let batches = collect(physical_plan.execute(0, task_ctx)?).await?; + let batches = + collect(execute_plan(Arc::clone(&physical_plan), 0, task_ctx)?).await?; // Get string representation of the plan assert_snapshot!(displayable(physical_plan.as_ref()).indent(true), @r#" diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index d6d5f4fdd2a67..5e31ccf4a1e3b 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -24,7 +24,9 @@ use std::task::{Context, Poll}; use super::utils::create_schema; use crate::execution_plan::EmissionType; -use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use crate::metrics::BaselineMetrics; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, window_equivalence_properties, @@ -44,6 +46,8 @@ use datafusion_common::stats::Precision; use datafusion_common::utils::{evaluate_partition_ranges, transpose}; use datafusion_common::{Result, assert_eq_or_internal_err}; use datafusion_execution::TaskContext; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_expr_common::sort_expr::{ OrderingRequirements, PhysicalSortExpr, }; @@ -59,8 +63,6 @@ pub struct WindowAggExec { window_expr: Vec>, /// Schema after the window is run schema: SchemaRef, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, /// Partition by indices that defines preset for existing ordering // see `get_ordered_partition_by_indices` for more details. ordered_partition_by_indices: Vec, @@ -68,6 +70,9 @@ pub struct WindowAggExec { cache: PlanProperties, /// If `can_partition` is false, partition_keys is always empty. can_repartition: bool, + /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet, } impl WindowAggExec { @@ -87,10 +92,11 @@ impl WindowAggExec { input, window_expr, schema, - metrics: ExecutionPlanMetricsSet::new(), ordered_partition_by_indices, cache, can_repartition, + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), }) } @@ -273,19 +279,26 @@ impl ExecutionPlan for WindowAggExec { &self, partition: usize, context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { - let input = self.input.execute(partition, context)?; + #[cfg(not(feature = "stateless_plan"))] + #[expect(unused)] + let state = (); + use crate::{execute_input, plan_metrics}; + + let input = execute_input!(0, self.input, partition, context, state)?; let stream = Box::pin(WindowAggStream::new( Arc::clone(&self.schema), self.window_expr.clone(), input, - BaselineMetrics::new(&self.metrics, partition), + BaselineMetrics::new(plan_metrics!(self, state), partition), self.partition_by_sort_keys()?, self.ordered_partition_by_indices.clone(), )?); Ok(stream) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index ba7c98c26480b..1f9a2e6bc8e7a 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -23,7 +23,8 @@ use std::sync::{Arc, Mutex}; use crate::coop::cooperative; use crate::execution_plan::{Boundedness, EmissionType, SchedulingType}; use crate::memory::MemoryStream; -use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +#[cfg(feature = "stateless_plan")] +use crate::state::PlanStateNode; use crate::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, SendableRecordBatchStream, Statistics, @@ -34,6 +35,8 @@ use arrow::record_batch::RecordBatch; use datafusion_common::{Result, assert_eq_or_internal_err, internal_datafusion_err}; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::MemoryReservation; +#[cfg(not(feature = "stateless_plan"))] +use datafusion_execution::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; /// A vector of record batches with a memory reservation. @@ -100,12 +103,14 @@ pub struct WorkTableExec { name: String, /// The schema of the stream schema: SchemaRef, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, /// The work table + #[cfg(not(feature = "stateless_plan"))] work_table: Arc, /// Execution metrics + #[cfg(not(feature = "stateless_plan"))] metrics: ExecutionPlanMetricsSet, - /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanProperties, } impl WorkTableExec { @@ -115,9 +120,11 @@ impl WorkTableExec { Self { name, schema, - metrics: ExecutionPlanMetricsSet::new(), - work_table: Arc::new(WorkTable::new()), cache, + #[cfg(not(feature = "stateless_plan"))] + work_table: Arc::new(WorkTable::new()), + #[cfg(not(feature = "stateless_plan"))] + metrics: ExecutionPlanMetricsSet::new(), } } @@ -189,6 +196,7 @@ impl ExecutionPlan for WorkTableExec { &self, partition: usize, _context: Arc, + #[cfg(feature = "stateless_plan")] state: &Arc, ) -> Result { // WorkTable streams must be the plan base. assert_eq_or_internal_err!( @@ -196,7 +204,20 @@ impl ExecutionPlan for WorkTableExec { 0, "WorkTableExec got an invalid partition {partition} (expected 0)" ); - let batch = self.work_table.take()?; + + #[cfg(feature = "stateless_plan")] + let Some(work_table) = &state.work_table() else { + use datafusion_common::internal_err; + + return internal_err!( + "work table is not found in one of the parent plan node" + ); + }; + + #[cfg(not(feature = "stateless_plan"))] + let work_table = &self.work_table; + + let batch = work_table.take()?; let stream = MemoryStream::try_new(batch.batches, Arc::clone(&self.schema), None)? @@ -204,6 +225,7 @@ impl ExecutionPlan for WorkTableExec { Ok(Box::pin(cooperative(stream))) } + #[cfg(not(feature = "stateless_plan"))] fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } @@ -216,6 +238,7 @@ impl ExecutionPlan for WorkTableExec { Ok(Statistics::new_unknown(&self.schema())) } + #[cfg(not(feature = "stateless_plan"))] /// Injects run-time state into this `WorkTableExec`. /// /// The only state this node currently understands is an [`Arc`]. diff --git a/datafusion/pruning/src/pruning_predicate.rs b/datafusion/pruning/src/pruning_predicate.rs index b5b8267d7f93f..48dfe252bd915 100644 --- a/datafusion/pruning/src/pruning_predicate.rs +++ b/datafusion/pruning/src/pruning_predicate.rs @@ -2987,7 +2987,7 @@ mod tests { .map(|c| Arc::new(c.clone()) as Arc) .collect_vec(); let dynamic_phys_expr = - Arc::new(DynamicFilterPhysicalExpr::new(children, phys_expr)) + Arc::new(DynamicFilterPhysicalExpr::new(phys_expr, children)) as Arc; // Simulate the partition value substitution that would happen in ParquetOpener let remapped_expr = dynamic_phys_expr @@ -4682,7 +4682,7 @@ mod tests { true, // s1 ["AB", "A\u{10ffff}\u{10ffff}\u{10ffff}"] ==> some rows could pass (must keep) true, - // s1 ["A\u{10ffff}\u{10ffff}", "A\u{10ffff}\u{10ffff}"] ==> no row match. (min, max) maybe truncate + // s1 ["A\u{10ffff}\u{10ffff}", "A\u{10ffff}\u{10ffff}"] ==> no row match. (min, max) maybe truncate // original (min, max) maybe ("A\u{10ffff}\u{10ffff}\u{10ffff}", "A\u{10ffff}\u{10ffff}\u{10ffff}\u{10ffff}") true, ];