From 7e009ff2db1b2fbd2760923497f18b2a7c46557e Mon Sep 17 00:00:00 2001 From: Albert Skalt Date: Mon, 22 Dec 2025 21:44:19 +0300 Subject: [PATCH 1/2] feat: support stateless physical plans This patch introduces the stateless physical plan feature. Currently, the physical-plan crate is fully supported. This feature allows for the reuse of physical plans and their concurrent execution. The feature is implemented by adding a separate Cargo feature named "stateless_plan". The implementation consists of several parts: * State tree. With the "stateless_plan" feature enabled, the plans themselves do not store state. The state is stored in a separate tree composed of PlanStateNodes, which is built lazily during plan execution. Each node of the tree stores not only the shared state of the plan but also its metrics. The shape of the state tree matches the shape of the execution plan tree. * Metrics Metrics are stored in the nodes of the state tree and can be accessed after plan execution. Support is provided for performing EXPLAIN using the state. * Dynamic Filters In the case of stateless plans, dynamic filters cannot simply be stored inside the plans, as the same plan can be executed concurrently. To overcome this, a dynamic filter is split into two parts: a planning-time version and an execution-time version. The plans contain the planning-time version, which is transformed into the execution version during the execution phase and then passed from parent nodes to child nodes using the state tree. * WorkTable Instead of explicitly injecting the WorkTable into nodes, RecursiveExec exposes the WorkTable in the state stored within the State Tree. Then, a node interested in obtaining the WorkTable traverses up the State Tree and thus retrieves the current WorkTable. Planned following work: - Support stateless plan for all other DataFusion crates. - Enable running tests with this feature in CI. - Deprecate stateful plans to eventually transition completely to the stateless version. - Add `fmt_as_with_state` to allow plans to include state-specific details in the EXPLAIN output, such as dynamic filters. Closes https://github.com/apache/datafusion/issues/19351 --- datafusion/datasource-parquet/src/opener.rs | 6 +- datafusion/execution/src/metrics/mod.rs | 16 +- .../src/expressions/dynamic_filters.rs | 311 +++++++++---- .../physical-expr/src/expressions/mod.rs | 2 +- datafusion/physical-plan/Cargo.toml | 3 + .../src/aggregates/group_values/metrics.rs | 31 +- .../physical-plan/src/aggregates/mod.rs | 270 +++++++++-- .../src/aggregates/no_grouping.rs | 26 +- .../physical-plan/src/aggregates/row_hash.rs | 47 +- .../src/aggregates/topk_stream.rs | 15 +- datafusion/physical-plan/src/analyze.rs | 44 +- datafusion/physical-plan/src/async_func.rs | 17 +- .../physical-plan/src/coalesce_batches.rs | 24 +- .../physical-plan/src/coalesce_partitions.rs | 55 ++- datafusion/physical-plan/src/coop.rs | 10 +- datafusion/physical-plan/src/display.rs | 381 ++++++++++++---- .../physical-plan/src/dynamic_filter.rs | 88 ++++ datafusion/physical-plan/src/empty.rs | 12 +- .../physical-plan/src/execution_plan.rs | 275 ++++++++++- datafusion/physical-plan/src/explain.rs | 3 + datafusion/physical-plan/src/filter.rs | 29 +- .../physical-plan/src/joins/cross_join.rs | 72 ++- .../physical-plan/src/joins/hash_join/exec.rs | 317 ++++++++----- .../src/joins/nested_loop_join.rs | 92 ++-- .../piecewise_merge_join/classic_join.rs | 3 +- .../src/joins/piecewise_merge_join/exec.rs | 109 +++-- .../src/joins/sort_merge_join/exec.rs | 44 +- .../src/joins/sort_merge_join/tests.rs | 103 +++-- .../src/joins/symmetric_hash_join.rs | 31 +- .../physical-plan/src/joins/test_utils.rs | 34 +- datafusion/physical-plan/src/lib.rs | 9 +- datafusion/physical-plan/src/limit.rs | 48 +- datafusion/physical-plan/src/memory.rs | 34 +- .../physical-plan/src/placeholder_row.rs | 22 +- datafusion/physical-plan/src/projection.rs | 31 +- .../physical-plan/src/recursive_query.rs | 148 +++++- .../physical-plan/src/repartition/mod.rs | 303 ++++++------ .../physical-plan/src/sorts/partial_sort.rs | 38 +- datafusion/physical-plan/src/sorts/sort.rs | 192 ++++++-- .../src/sorts/sort_preserving_merge.rs | 64 ++- datafusion/physical-plan/src/state.rs | 431 ++++++++++++++++++ datafusion/physical-plan/src/stream.rs | 60 ++- datafusion/physical-plan/src/streaming.rs | 19 +- datafusion/physical-plan/src/test.rs | 84 +++- datafusion/physical-plan/src/test/exec.rs | 16 +- datafusion/physical-plan/src/topk/mod.rs | 6 +- datafusion/physical-plan/src/union.rs | 48 +- datafusion/physical-plan/src/unnest.rs | 26 +- .../src/windows/bounded_window_agg_exec.rs | 29 +- .../src/windows/window_agg_exec.rs | 25 +- datafusion/physical-plan/src/work_table.rs | 35 +- datafusion/pruning/src/pruning_predicate.rs | 4 +- 52 files changed, 3232 insertions(+), 910 deletions(-) create mode 100644 datafusion/physical-plan/src/dynamic_filter.rs create mode 100644 datafusion/physical-plan/src/state.rs 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..54f0c1a46b531 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" 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..8abc373bd5809 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; @@ -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, ]; From 4275e0264a61fac347530b6a393ef7114a2f8767 Mon Sep 17 00:00:00 2001 From: Albert Skalt Date: Mon, 29 Dec 2025 18:00:15 +0300 Subject: [PATCH 2/2] add physical plan re-use benchmark --- datafusion/physical-plan/Cargo.toml | 4 + .../physical-plan/benches/plan_reuse.rs | 192 ++++++++++++++++++ datafusion/physical-plan/src/test.rs | 2 +- 3 files changed, 197 insertions(+), 1 deletion(-) create mode 100644 datafusion/physical-plan/benches/plan_reuse.rs diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 54f0c1a46b531..a73420928330a 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -92,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/test.rs b/datafusion/physical-plan/src/test.rs index 8abc373bd5809..9ff8f19ff2158 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -148,7 +148,7 @@ impl ExecutionPlan for TestMemoryExec { self: Arc, _: Vec>, ) -> Result> { - unimplemented!() + Ok(self) } fn repartitioned(