From c358487c6fff82122a7fa0c6ebeb0a675f0b1b9c Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 31 Mar 2025 19:12:25 +0800 Subject: [PATCH 01/26] save --- datafusion/datasource/src/source.rs | 17 +++++++++++++++++ datafusion/physical-plan/src/execution_plan.rs | 10 ++++++++++ datafusion/physical-plan/src/repartition/mod.rs | 4 ++++ datafusion/physical-plan/src/sorts/sort.rs | 4 ++++ datafusion/physical-plan/src/union.rs | 11 +++++++++++ 5 files changed, 46 insertions(+) diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 6abe3c329869e..ad6b8d3a719b6 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -201,6 +201,23 @@ impl ExecutionPlan for DataSourceExec { self.data_source.statistics() } + fn statistics_by_partition(&self) -> datafusion_common::Result> { + let mut statistics = vec![ + Statistics::new_unknown(&self.schema()); + self.properties().partitioning.partition_count() + ]; + if let Some(file_config) = + self.data_source.as_any().downcast_ref::() + { + for (idx, file_group) in file_config.file_groups.iter().enumerate() { + if let Some(stat) = file_group.statistics() { + statistics[idx] = stat.clone(); + } + } + } + Ok(statistics) + } + fn with_fetch(&self, limit: Option) -> Option> { let data_source = self.data_source.with_fetch(limit)?; let cache = self.cache.clone(); diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 2b6eac7be0675..ab69ac9af32a2 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -430,6 +430,16 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { Ok(Statistics::new_unknown(&self.schema())) } + /// Returns statistics for each partition of this `ExecutionPlan` node. + /// If statistics are not available, returns an array of + /// [`Statistics::new_unknown`] for each partition. + fn statistics_by_partition(&self) -> Result> { + Ok(vec![ + Statistics::new_unknown(&self.schema()); + self.properties().partitioning.partition_count() + ]) + } + /// Returns `true` if a limit can be safely pushed down through this /// `ExecutionPlan` node. /// diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index c480fc2abaa1a..2bbaddd8fca07 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -691,6 +691,10 @@ impl ExecutionPlan for RepartitionExec { self.input.statistics() } + fn statistics_by_partition(&self) -> Result> { + todo!() + } + fn cardinality_effect(&self) -> CardinalityEffect { CardinalityEffect::Equal } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 9d0f34cc7f0fd..3bcd7050da486 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1300,6 +1300,10 @@ impl ExecutionPlan for SortExec { Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } + fn statistics_by_partition(&self) -> Result> { + todo!() + } + fn with_fetch(&self, limit: Option) -> Option> { Some(Arc::new(SortExec::with_fetch(self, limit))) } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 2b666093f29e0..7225b1eee4dc6 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -270,6 +270,17 @@ impl ExecutionPlan for UnionExec { .unwrap_or_else(|| Statistics::new_unknown(&self.schema()))) } + fn statistics_by_partition(&self) -> Result> { + Ok(self + .inputs + .iter() + .map(|child| child.statistics_by_partition()) + .collect::>>()? + .into_iter() + .flatten() + .collect()) + } + fn benefits_from_input_partitioning(&self) -> Vec { vec![false; self.children().len()] } From 47312d76f1382fd101f2d3403020df0651aa3cca Mon Sep 17 00:00:00 2001 From: xudong963 Date: Tue, 1 Apr 2025 19:14:49 +0800 Subject: [PATCH 02/26] save --- .../datasource/physical_plan/arrow_file.rs | 5 +++ datafusion/datasource-avro/src/source.rs | 4 ++ datafusion/datasource-csv/src/source.rs | 4 ++ .../physical-plan/src/coalesce_batches.rs | 4 ++ .../physical-plan/src/coalesce_partitions.rs | 4 ++ datafusion/physical-plan/src/filter.rs | 32 ++++++++++++--- .../physical-plan/src/joins/cross_join.rs | 25 ++++++++++++ .../physical-plan/src/joins/hash_join.rs | 4 +- .../src/joins/nested_loop_join.rs | 4 +- .../src/joins/sort_merge_join.rs | 4 +- datafusion/physical-plan/src/joins/utils.rs | 7 +--- datafusion/physical-plan/src/limit.rs | 4 ++ datafusion/physical-plan/src/sorts/sort.rs | 13 +++++- .../src/windows/bounded_window_agg_exec.rs | 40 +++++++++++++------ 14 files changed, 123 insertions(+), 31 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index f0a1f94d87e1f..ba23cf4095398 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -190,6 +190,11 @@ impl ExecutionPlan for ArrowExec { fn statistics(&self) -> Result { self.inner.statistics() } + + fn statistics_by_partition(&self) -> Result> { + self.inner.statistics_by_partition() + } + fn fetch(&self) -> Option { self.inner.fetch() } diff --git a/datafusion/datasource-avro/src/source.rs b/datafusion/datasource-avro/src/source.rs index ce3722e7b11ee..5c662511929c7 100644 --- a/datafusion/datasource-avro/src/source.rs +++ b/datafusion/datasource-avro/src/source.rs @@ -141,6 +141,10 @@ impl ExecutionPlan for AvroExec { self.inner.statistics() } + fn statistics_by_partition(&self) -> Result> { + self.inner.statistics_by_partition() + } + fn metrics(&self) -> Option { self.inner.metrics() } diff --git a/datafusion/datasource-csv/src/source.rs b/datafusion/datasource-csv/src/source.rs index f5d45cd3fc881..22fbb288c0396 100644 --- a/datafusion/datasource-csv/src/source.rs +++ b/datafusion/datasource-csv/src/source.rs @@ -381,6 +381,10 @@ impl ExecutionPlan for CsvExec { self.inner.statistics() } + fn statistics_by_partition(&self) -> Result> { + self.inner.statistics_by_partition() + } + fn metrics(&self) -> Option { self.inner.metrics() } diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index faab5fdc5eb6c..314e5548b36a9 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -199,6 +199,10 @@ impl ExecutionPlan for CoalesceBatchesExec { Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } + fn statistics_by_partition(&self) -> Result> { + Ok(vec![self.statistics()?]) + } + fn with_fetch(&self, limit: Option) -> Option> { Some(Arc::new(CoalesceBatchesExec { input: Arc::clone(&self.input), diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 715dd159e7e8c..1de838e830c23 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -199,6 +199,10 @@ impl ExecutionPlan for CoalescePartitionsExec { Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } + fn statistics_by_partition(&self) -> Result> { + Ok(vec![self.statistics()?]) + } + fn supports_limit_pushdown(&self) -> bool { true } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 95fa67025e90d..f179caddc0785 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -174,12 +174,11 @@ impl FilterExec { /// Calculates `Statistics` for `FilterExec`, by applying selectivity (either default, or estimated) to input statistics. fn statistics_helper( - input: &Arc, + schema: SchemaRef, + input_stats: Statistics, predicate: &Arc, default_selectivity: u8, ) -> Result { - let input_stats = input.statistics()?; - let schema = input.schema(); if !check_support(predicate, &schema) { let selectivity = default_selectivity as f64 / 100.0; let mut stats = input_stats.to_inexact(); @@ -193,7 +192,7 @@ impl FilterExec { let num_rows = input_stats.num_rows; let total_byte_size = input_stats.total_byte_size; let input_analysis_ctx = AnalysisContext::try_from_statistics( - &input.schema(), + &schema, &input_stats.column_statistics, )?; @@ -260,7 +259,12 @@ impl FilterExec { ) -> Result { // Combine the equal predicates with the input equivalence properties // to construct the equivalence properties: - let stats = Self::statistics_helper(input, predicate, default_selectivity)?; + let stats = Self::statistics_helper( + input.schema(), + input.statistics()?, + predicate, + default_selectivity, + )?; let mut eq_properties = input.equivalence_properties().clone(); let (equal_pairs, _) = collect_columns_from_predicate(predicate); for (lhs, rhs) in equal_pairs { @@ -401,13 +405,29 @@ impl ExecutionPlan for FilterExec { /// predicate's selectivity value can be determined for the incoming data. fn statistics(&self) -> Result { let stats = Self::statistics_helper( - &self.input, + self.schema(), + self.input().statistics()?, self.predicate(), self.default_selectivity, )?; Ok(stats.project(self.projection.as_ref())) } + fn statistics_by_partition(&self) -> Result> { + let input_stats = self.input.statistics_by_partition()?; + let mut stats = Vec::with_capacity(input_stats.len()); + for input_stat in input_stats { + let stat = Self::statistics_helper( + self.schema(), + input_stat, + self.predicate(), + self.default_selectivity, + )?; + stats.push(stat.project(self.projection.as_ref())); + } + Ok(stats) + } + fn cardinality_effect(&self) -> CardinalityEffect { CardinalityEffect::LowerEqual } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 8dd1addff15ce..f8e101f61dd61 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -343,6 +343,31 @@ impl ExecutionPlan for CrossJoinExec { )) } + fn statistics_by_partition(&self) -> Result> { + todo!() + /* + let left_stats = self.left.statistics_by_partition()?; + // Summarize the `left_stats` + let statistics = + compute_summary_statistics(file_group.iter(), &file_schema, |file| { + file.statistics.as_ref().map(|stats| stats.as_ref()) + }); + let right_stats = self.right.statistics_by_partition()?; + + if left_stats.is_empty() || right_stats.is_empty() { + return Ok(vec![]); + } + + let mut stats = Vec::new(); + for left in left_stats.iter() { + for right in right_stats.iter() { + stats.push(stats_cartesian_product(left.clone(), right.clone())); + } + } + Ok(stats) + */ + } + /// Tries to swap the projection with its input [`CrossJoinExec`]. If it can be done, /// it returns the new swapped version having the [`CrossJoinExec`] as the top plan. /// Otherwise, it returns None. diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index e8904db0f3eaf..50282e056777d 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -883,8 +883,8 @@ impl ExecutionPlan for HashJoinExec { // There are some special cases though, for example: // - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)` let stats = estimate_join_statistics( - Arc::clone(&self.left), - Arc::clone(&self.right), + self.left.statistics()?, + self.right.statistics()?, self.on.clone(), &self.join_type, &self.join_schema, diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index b902795950966..94b6725bb19cf 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -568,8 +568,8 @@ impl ExecutionPlan for NestedLoopJoinExec { fn statistics(&self) -> Result { estimate_join_statistics( - Arc::clone(&self.left), - Arc::clone(&self.right), + self.left.statistics()?, + self.right.statistics()?, vec![], &self.join_type, &self.join_schema, diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 89f2e3c911f89..28612540bce5b 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -518,8 +518,8 @@ impl ExecutionPlan for SortMergeJoinExec { // There are some special cases though, for example: // - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)` estimate_join_statistics( - Arc::clone(&self.left), - Arc::clone(&self.right), + self.left.statistics()?, + self.right.statistics()?, self.on.clone(), &self.join_type, &self.schema, diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 5516f172d5101..ce1860d20565c 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -403,15 +403,12 @@ struct PartialJoinStatistics { /// Estimate the statistics for the given join's output. pub(crate) fn estimate_join_statistics( - left: Arc, - right: Arc, + left_stats: Statistics, + right_stats: Statistics, on: JoinOn, join_type: &JoinType, schema: &Schema, ) -> Result { - let left_stats = left.statistics()?; - let right_stats = right.statistics()?; - let join_stats = estimate_join_cardinality(join_type, left_stats, right_stats, &on); let (num_rows, column_statistics) = match join_stats { Some(stats) => (Precision::Inexact(stats.num_rows), stats.column_statistics), diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 89cf47a6d6508..78bca3c794413 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -202,6 +202,10 @@ impl ExecutionPlan for GlobalLimitExec { ) } + fn statistics_by_partition(&self) -> Result> { + Ok(vec![self.statistics()?]) + } + fn fetch(&self) -> Option { self.fetch } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 3bcd7050da486..1334f3b091ecb 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1301,7 +1301,18 @@ impl ExecutionPlan for SortExec { } fn statistics_by_partition(&self) -> Result> { - todo!() + let input_stats = self.input.statistics_by_partition()?; + let mut stats = Vec::with_capacity(input_stats.len()); + for stat in input_stats { + stats.push(Statistics::with_fetch( + stat, + self.schema(), + self.fetch, + 0, + 1, + )?); + } + Ok(stats) } fn with_fetch(&self, limit: Option) -> Option> { 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 92138bf6a7a1a..af6c5f29653ee 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -226,6 +226,23 @@ impl BoundedWindowAggExec { .unwrap_or_else(Vec::new) } } + + fn statistics_helper(&self, statistics: Statistics) -> Result { + let win_cols = self.window_expr.len(); + let input_cols = self.input.schema().fields().len(); + // TODO stats: some windowing function will maintain invariants such as min, max... + let mut column_statistics = Vec::with_capacity(win_cols + input_cols); + // copy stats of the input to the beginning of the schema. + column_statistics.extend(statistics.column_statistics); + for _ in 0..win_cols { + column_statistics.push(ColumnStatistics::new_unknown()) + } + Ok(Statistics { + num_rows: statistics.num_rows, + column_statistics, + total_byte_size: Precision::Absent, + }) + } } impl DisplayAs for BoundedWindowAggExec { @@ -344,20 +361,17 @@ impl ExecutionPlan for BoundedWindowAggExec { fn statistics(&self) -> Result { let input_stat = self.input.statistics()?; - let win_cols = self.window_expr.len(); - let input_cols = self.input.schema().fields().len(); - // TODO stats: some windowing function will maintain invariants such as min, max... - let mut column_statistics = Vec::with_capacity(win_cols + input_cols); - // copy stats of the input to the beginning of the schema. - column_statistics.extend(input_stat.column_statistics); - for _ in 0..win_cols { - column_statistics.push(ColumnStatistics::new_unknown()) + self.statistics_helper(input_stat) + } + + fn statistics_by_partition(&self) -> Result> { + let input_stats = self.input.statistics_by_partition()?; + let mut output_stats = Vec::with_capacity(input_stats.len()); + for stat in input_stats { + let output_stat = self.statistics_helper(stat.clone())?; + output_stats.push(output_stat); } - Ok(Statistics { - num_rows: input_stat.num_rows, - column_statistics, - total_byte_size: Precision::Absent, - }) + Ok(output_stats) } } From 2f94bdaf62ea7ad8670f8f33aca0ae92b51813dc Mon Sep 17 00:00:00 2001 From: xudong963 Date: Wed, 2 Apr 2025 11:33:47 +0800 Subject: [PATCH 03/26] save --- .../core/src/datasource/listing/table.rs | 1 + datafusion/datasource/src/statistics.rs | 3 + .../physical-plan/src/joins/cross_join.rs | 22 ++- datafusion/physical-plan/src/lib.rs | 1 + datafusion/physical-plan/src/limit.rs | 16 ++ datafusion/physical-plan/src/projection.rs | 14 ++ .../src/sorts/sort_preserving_merge.rs | 4 + datafusion/physical-plan/src/statistics.rs | 151 ++++++++++++++++++ 8 files changed, 200 insertions(+), 12 deletions(-) create mode 100644 datafusion/physical-plan/src/statistics.rs diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 94668dee16132..f42db18d10545 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -58,6 +58,7 @@ use datafusion_common::stats::Precision; use datafusion_datasource::compute_all_files_statistics; use datafusion_datasource::file_groups::FileGroup; use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_plan::statistics::add_row_stats; use futures::{future, stream, Stream, StreamExt, TryStreamExt}; use itertools::Itertools; use object_store::ObjectStore; diff --git a/datafusion/datasource/src/statistics.rs b/datafusion/datasource/src/statistics.rs index 8a04d77b273d4..5b44860091256 100644 --- a/datafusion/datasource/src/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -36,6 +36,9 @@ use datafusion_common::stats::Precision; use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; use datafusion_physical_expr::{expressions::Column, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::statistics::{ + add_row_stats, compute_summary_statistics, set_max_if_greater, set_min_if_lesser, +}; use datafusion_physical_plan::{ColumnStatistics, Statistics}; /// A normalized representation of file min/max statistics that allows for efficient sorting & comparison. diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index f8e101f61dd61..93fce93fff1d3 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -46,6 +46,7 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; +use crate::statistics::compute_summary_statistics; use async_trait::async_trait; use futures::{ready, Stream, StreamExt, TryStreamExt}; @@ -344,28 +345,25 @@ impl ExecutionPlan for CrossJoinExec { } fn statistics_by_partition(&self) -> Result> { - todo!() - /* let left_stats = self.left.statistics_by_partition()?; - // Summarize the `left_stats` - let statistics = - compute_summary_statistics(file_group.iter(), &file_schema, |file| { - file.statistics.as_ref().map(|stats| stats.as_ref()) - }); let right_stats = self.right.statistics_by_partition()?; if left_stats.is_empty() || right_stats.is_empty() { return Ok(vec![]); } + // Summarize the `left_stats` + let statistics = compute_summary_statistics( + left_stats.iter(), + self.schema.fields().len(), + |stats| Some(stats), + ); + let mut stats = Vec::new(); - for left in left_stats.iter() { - for right in right_stats.iter() { - stats.push(stats_cartesian_product(left.clone(), right.clone())); - } + for right in right_stats.iter() { + stats.push(stats_cartesian_product(statistics.clone(), right.clone())); } Ok(stats) - */ } /// Tries to swap the projection with its input [`CrossJoinExec`]. If it can be done, diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index a1862554b303e..4493569ae6ffd 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -92,5 +92,6 @@ pub mod udaf { } pub mod coalesce; +pub mod statistics; #[cfg(test)] pub mod test; diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 78bca3c794413..b66e1c53f2041 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -347,6 +347,22 @@ impl ExecutionPlan for LocalLimitExec { ) } + fn statistics_by_partition(&self) -> Result> { + let input_stats = self.input.statistics_by_partition()?; + let mut stats = Vec::with_capacity(input_stats.len()); + for input_stat in input_stats { + let stat = Statistics::with_fetch( + input_stat, + self.schema(), + Some(self.fetch), + 0, + 1, + )?; + stats.push(stat); + } + Ok(stats) + } + fn fetch(&self) -> Option { Some(self.fetch) } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 72934c74446eb..da67766078de4 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -251,6 +251,20 @@ impl ExecutionPlan for ProjectionExec { )) } + fn statistics_by_partition(&self) -> Result> { + let input_stats = self.input.statistics_by_partition()?; + let mut stats = Vec::with_capacity(input_stats.len()); + for input_stat in input_stats { + let stat = stats_projection( + input_stat.clone(), + self.expr.iter().map(|(e, _)| Arc::clone(e)), + Arc::clone(&self.schema), + ); + stats.push(stat); + } + Ok(stats) + } + fn supports_limit_pushdown(&self) -> bool { true } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index b987dff36441d..a1cfb3233751f 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -346,6 +346,10 @@ impl ExecutionPlan for SortPreservingMergeExec { self.input.statistics() } + fn statistics_by_partition(&self) -> Result> { + Ok(vec![self.statistics()?]) + } + fn supports_limit_pushdown(&self) -> bool { true } diff --git a/datafusion/physical-plan/src/statistics.rs b/datafusion/physical-plan/src/statistics.rs new file mode 100644 index 0000000000000..3d70d907219d4 --- /dev/null +++ b/datafusion/physical-plan/src/statistics.rs @@ -0,0 +1,151 @@ +// 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. + +//! Defines the cross join plan for loading the left side of the cross join +//! and producing batches in parallel for the right partitions + +use datafusion_common::stats::Precision; +use datafusion_common::{ColumnStatistics, ScalarValue, Statistics}; +use std::mem; + +/// Generic function to compute statistics across multiple items that have statistics +pub fn compute_summary_statistics( + items: I, + column_count: usize, + stats_extractor: impl Fn(&T) -> Option<&Statistics>, +) -> Statistics +where + I: IntoIterator, +{ + let mut col_stats_set = vec![ColumnStatistics::default(); column_count]; + let mut num_rows = Precision::::Absent; + let mut total_byte_size = Precision::::Absent; + + for (idx, item) in items.into_iter().enumerate() { + if let Some(item_stats) = stats_extractor(&item) { + if idx == 0 { + // First item, set values directly + num_rows = item_stats.num_rows; + total_byte_size = item_stats.total_byte_size; + for (index, column_stats) in + item_stats.column_statistics.iter().enumerate() + { + col_stats_set[index].null_count = column_stats.null_count; + col_stats_set[index].max_value = column_stats.max_value.clone(); + col_stats_set[index].min_value = column_stats.min_value.clone(); + col_stats_set[index].sum_value = column_stats.sum_value.clone(); + } + continue; + } + + // Accumulate statistics for subsequent items + num_rows = add_row_stats(item_stats.num_rows, num_rows); + total_byte_size = add_row_stats(item_stats.total_byte_size, total_byte_size); + + for (item_col_stats, col_stats) in item_stats + .column_statistics + .iter() + .zip(col_stats_set.iter_mut()) + { + col_stats.null_count = + add_row_stats(item_col_stats.null_count, col_stats.null_count); + set_max_if_greater(&item_col_stats.max_value, &mut col_stats.max_value); + set_min_if_lesser(&item_col_stats.min_value, &mut col_stats.min_value); + col_stats.sum_value = item_col_stats.sum_value.add(&col_stats.sum_value); + } + } + } + + Statistics { + num_rows, + total_byte_size, + column_statistics: col_stats_set, + } +} + +/// If the given value is numerically greater than the original maximum value, +/// return the new maximum value with appropriate exactness information. +pub fn set_max_if_greater( + max_nominee: &Precision, + max_value: &mut Precision, +) { + match (&max_value, max_nominee) { + (Precision::Exact(val1), Precision::Exact(val2)) if val1 < val2 => { + *max_value = max_nominee.clone(); + } + (Precision::Exact(val1), Precision::Inexact(val2)) + | (Precision::Inexact(val1), Precision::Inexact(val2)) + | (Precision::Inexact(val1), Precision::Exact(val2)) + if val1 < val2 => + { + *max_value = max_nominee.clone().to_inexact(); + } + (Precision::Exact(_), Precision::Absent) => { + let exact_max = mem::take(max_value); + *max_value = exact_max.to_inexact(); + } + (Precision::Absent, Precision::Exact(_)) => { + *max_value = max_nominee.clone().to_inexact(); + } + (Precision::Absent, Precision::Inexact(_)) => { + *max_value = max_nominee.clone(); + } + _ => {} + } +} + +/// If the given value is numerically lesser than the original minimum value, +/// return the new minimum value with appropriate exactness information. +pub fn set_min_if_lesser( + min_nominee: &Precision, + min_value: &mut Precision, +) { + match (&min_value, min_nominee) { + (Precision::Exact(val1), Precision::Exact(val2)) if val1 > val2 => { + *min_value = min_nominee.clone(); + } + (Precision::Exact(val1), Precision::Inexact(val2)) + | (Precision::Inexact(val1), Precision::Inexact(val2)) + | (Precision::Inexact(val1), Precision::Exact(val2)) + if val1 > val2 => + { + *min_value = min_nominee.clone().to_inexact(); + } + (Precision::Exact(_), Precision::Absent) => { + let exact_min = mem::take(min_value); + *min_value = exact_min.to_inexact(); + } + (Precision::Absent, Precision::Exact(_)) => { + *min_value = min_nominee.clone().to_inexact(); + } + (Precision::Absent, Precision::Inexact(_)) => { + *min_value = min_nominee.clone(); + } + _ => {} + } +} + +pub fn add_row_stats( + file_num_rows: Precision, + num_rows: Precision, +) -> Precision { + match (file_num_rows, &num_rows) { + (Precision::Absent, _) => num_rows.to_inexact(), + (lhs, Precision::Absent) => lhs.to_inexact(), + (lhs, rhs) => lhs.add(rhs), + } +} From 942af19b26dd37e353c8b7544637c4a993d6c2f3 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Wed, 2 Apr 2025 11:48:38 +0800 Subject: [PATCH 04/26] functional way --- datafusion/physical-plan/src/filter.rs | 25 ++++++++++--------- .../physical-plan/src/joins/cross_join.rs | 11 ++++---- datafusion/physical-plan/src/limit.rs | 20 ++++++--------- datafusion/physical-plan/src/projection.rs | 23 +++++++++-------- .../physical-plan/src/repartition/mod.rs | 4 --- datafusion/physical-plan/src/sorts/sort.rs | 17 ++++--------- .../src/windows/bounded_window_agg_exec.rs | 12 ++++----- 7 files changed, 47 insertions(+), 65 deletions(-) diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index f179caddc0785..8378974be5d9b 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -414,18 +414,19 @@ impl ExecutionPlan for FilterExec { } fn statistics_by_partition(&self) -> Result> { - let input_stats = self.input.statistics_by_partition()?; - let mut stats = Vec::with_capacity(input_stats.len()); - for input_stat in input_stats { - let stat = Self::statistics_helper( - self.schema(), - input_stat, - self.predicate(), - self.default_selectivity, - )?; - stats.push(stat.project(self.projection.as_ref())); - } - Ok(stats) + self.input + .statistics_by_partition()? + .into_iter() + .map(|input_stat| { + Self::statistics_helper( + self.schema(), + input_stat, + self.predicate(), + self.default_selectivity, + ) + .map(|stat| stat.project(self.projection.as_ref())) + }) + .collect() } fn cardinality_effect(&self) -> CardinalityEffect { diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 93fce93fff1d3..22f67d5baf627 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -354,16 +354,15 @@ impl ExecutionPlan for CrossJoinExec { // Summarize the `left_stats` let statistics = compute_summary_statistics( - left_stats.iter(), + left_stats.into_iter(), self.schema.fields().len(), |stats| Some(stats), ); - let mut stats = Vec::new(); - for right in right_stats.iter() { - stats.push(stats_cartesian_product(statistics.clone(), right.clone())); - } - Ok(stats) + Ok(right_stats + .into_iter() + .map(|right| stats_cartesian_product(statistics.clone(), right)) + .collect()) } /// Tries to swap the projection with its input [`CrossJoinExec`]. If it can be done, diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index b66e1c53f2041..436b937f7086b 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -348,19 +348,13 @@ impl ExecutionPlan for LocalLimitExec { } fn statistics_by_partition(&self) -> Result> { - let input_stats = self.input.statistics_by_partition()?; - let mut stats = Vec::with_capacity(input_stats.len()); - for input_stat in input_stats { - let stat = Statistics::with_fetch( - input_stat, - self.schema(), - Some(self.fetch), - 0, - 1, - )?; - stats.push(stat); - } - Ok(stats) + self.input + .statistics_by_partition()? + .into_iter() + .map(|input_stat| { + Statistics::with_fetch(input_stat, self.schema(), Some(self.fetch), 0, 1) + }) + .collect() } fn fetch(&self) -> Option { diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index da67766078de4..629d9a2802af7 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -252,17 +252,18 @@ impl ExecutionPlan for ProjectionExec { } fn statistics_by_partition(&self) -> Result> { - let input_stats = self.input.statistics_by_partition()?; - let mut stats = Vec::with_capacity(input_stats.len()); - for input_stat in input_stats { - let stat = stats_projection( - input_stat.clone(), - self.expr.iter().map(|(e, _)| Arc::clone(e)), - Arc::clone(&self.schema), - ); - stats.push(stat); - } - Ok(stats) + Ok(self + .input + .statistics_by_partition()? + .into_iter() + .map(|input_stats| { + stats_projection( + input_stats, + self.expr.iter().map(|(e, _)| Arc::clone(e)), + Arc::clone(&self.schema), + ) + }) + .collect()) } fn supports_limit_pushdown(&self) -> bool { diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 2bbaddd8fca07..c480fc2abaa1a 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -691,10 +691,6 @@ impl ExecutionPlan for RepartitionExec { self.input.statistics() } - fn statistics_by_partition(&self) -> Result> { - todo!() - } - fn cardinality_effect(&self) -> CardinalityEffect { CardinalityEffect::Equal } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 1334f3b091ecb..bc729b9c18d16 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1301,18 +1301,11 @@ impl ExecutionPlan for SortExec { } fn statistics_by_partition(&self) -> Result> { - let input_stats = self.input.statistics_by_partition()?; - let mut stats = Vec::with_capacity(input_stats.len()); - for stat in input_stats { - stats.push(Statistics::with_fetch( - stat, - self.schema(), - self.fetch, - 0, - 1, - )?); - } - Ok(stats) + self.input + .statistics_by_partition()? + .into_iter() + .map(|stat| Statistics::with_fetch(stat, self.schema(), self.fetch, 0, 1)) + .collect() } fn with_fetch(&self, limit: Option) -> Option> { 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 af6c5f29653ee..139b1e3414a7e 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -365,13 +365,11 @@ impl ExecutionPlan for BoundedWindowAggExec { } fn statistics_by_partition(&self) -> Result> { - let input_stats = self.input.statistics_by_partition()?; - let mut output_stats = Vec::with_capacity(input_stats.len()); - for stat in input_stats { - let output_stat = self.statistics_helper(stat.clone())?; - output_stats.push(output_stat); - } - Ok(output_stats) + self.input + .statistics_by_partition()? + .into_iter() + .map(|stat| self.statistics_helper(stat)) + .collect() } } From 7b34cb4cf7b81644d7ffbe74f104a039208ab30c Mon Sep 17 00:00:00 2001 From: xudong963 Date: Wed, 2 Apr 2025 13:00:56 +0800 Subject: [PATCH 05/26] fix sort --- datafusion/physical-plan/src/joins/cross_join.rs | 9 ++++----- datafusion/physical-plan/src/sorts/sort.rs | 3 +++ 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 22f67d5baf627..3ad5276a547c6 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -353,11 +353,10 @@ impl ExecutionPlan for CrossJoinExec { } // Summarize the `left_stats` - let statistics = compute_summary_statistics( - left_stats.into_iter(), - self.schema.fields().len(), - |stats| Some(stats), - ); + let statistics = + compute_summary_statistics(left_stats, self.schema.fields().len(), |stats| { + Some(stats) + }); Ok(right_stats .into_iter() diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index bc729b9c18d16..e6f52496d3110 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1301,6 +1301,9 @@ impl ExecutionPlan for SortExec { } fn statistics_by_partition(&self) -> Result> { + if !self.preserve_partitioning() { + return Ok(vec![self.statistics()?]); + } self.input .statistics_by_partition()? .into_iter() From 8969cafc70cf4b79afcdfa35b98e2cf44ed0bae2 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Wed, 2 Apr 2025 16:45:50 +0800 Subject: [PATCH 06/26] adding test --- .../core/tests/physical_optimizer/mod.rs | 1 + .../partition_statistics.rs | 115 ++++++++++++++++++ 2 files changed, 116 insertions(+) create mode 100644 datafusion/core/tests/physical_optimizer/partition_statistics.rs diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 6643e7fd59b7a..43cec6d00d85c 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -29,3 +29,4 @@ mod push_down_filter; mod replace_with_order_preserving_variants; mod sanity_checker; mod test_utils; +mod partition_statistics; diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs new file mode 100644 index 0000000000000..8440976cbc68b --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -0,0 +1,115 @@ +// 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. + +#[cfg(test)] +mod test { + use std::sync::Arc; + use datafusion::datasource::listing::{ListingOptions, ListingTable, ListingTableConfig}; + use datafusion::execution::SessionStateBuilder; + use datafusion::prelude::SessionContext; + use datafusion_catalog::TableProvider; + use datafusion_common::config::ConfigOptions; + use datafusion_datasource::ListingTableUrl; + use datafusion_datasource::source::DataSourceExec; + use datafusion_datasource_parquet::ParquetFormat; + use datafusion_execution::config::SessionConfig; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use datafusion_physical_plan::ExecutionPlan; + + async fn generate_listing_table_with_statistics() -> Arc { + let testdata = datafusion::test_util::parquet_test_data(); + let filename = format!("{}/{}", testdata, "alltypes_tiny_pages.parquet"); + let table_path = ListingTableUrl::parse(filename).unwrap(); + let opt = ListingOptions::new(Arc::new(ParquetFormat::default())).with_collect_stat(true); + let rt = RuntimeEnvBuilder::new() + .build_arc() + .expect("could not build runtime environment"); + + let state = SessionContext::new_with_config_rt(SessionConfig::default(), rt).state(); + let schema = opt + .infer_schema( + &SessionStateBuilder::new().with_default_features().build(), + &table_path, + ) + .await + .unwrap(); + let config = ListingTableConfig::new(table_path.clone()) + .with_listing_options(opt.clone()) + .with_schema(schema); + let table = ListingTable::try_new(config).unwrap(); + let res= table.scan(&state, None, &[], None).await.unwrap(); + dbg!(&res.statistics().unwrap()); + dbg!(&res.statistics_by_partition().unwrap()); + let mut config = ConfigOptions::new(); + config.set("datafusion.optimizer.repartition_file_min_size", "10").unwrap(); + let res = res.repartitioned(5, &config).unwrap().unwrap(); + dbg!(&res.statistics_by_partition().unwrap()); + res + } + + #[tokio::test] + async fn test_statistics_by_partition_of_data_source() -> datafusion_common::Result<()> { + generate_listing_table_with_statistics().await; + Ok(()) + } + + #[test] + fn test_statistics_by_partition_of_projection() -> datafusion_common::Result<()> { + Ok(()) + } + + #[test] + fn test_statistics_by_partition_of_sort() -> datafusion_common::Result<()> { + Ok(()) + } + + #[test] + fn test_statistics_by_partition_of_filter() -> datafusion_common::Result<()> { + Ok(()) + } + + #[test] + fn test_statistics_by_partition_of_aggregate() -> datafusion_common::Result<()> { + Ok(()) + } + + #[test] + fn test_statistic_by_partition_of_cross_join() -> datafusion_common::Result<()> { + Ok(()) + } + + #[test] + fn test_statistic_by_partition_of_union() -> datafusion_common::Result<()> { + Ok(()) + } + + #[test] + fn test_statistic_by_partition_of_smp() -> datafusion_common::Result<()> { + Ok(()) + } + + #[test] + fn test_statistic_by_partition_of_limit() -> datafusion_common::Result<()> { + Ok(()) + } + + #[test] + fn test_statistic_by_partition_of_coalesce() -> datafusion_common::Result<()> { + Ok(()) + } + +} \ No newline at end of file From aaac604390eeb22e3f6b9661756fad222e75ff58 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Thu, 3 Apr 2025 15:56:04 +0800 Subject: [PATCH 07/26] add tests --- .../core/src/datasource/listing/table.rs | 2 +- .../core/tests/physical_optimizer/mod.rs | 2 +- .../partition_statistics.rs | 347 ++++++++++++++---- 3 files changed, 282 insertions(+), 69 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index f42db18d10545..25771fbcf40b6 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -740,7 +740,7 @@ impl ListingOptions { /// # Ok(()) /// # } /// ``` -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct ListingTable { table_paths: Vec, /// `file_schema` contains only the columns physically stored in the data files themselves. diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 43cec6d00d85c..c115a5253adcc 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -24,9 +24,9 @@ mod enforce_sorting; mod join_selection; mod limit_pushdown; mod limited_distinct_aggregation; +mod partition_statistics; mod projection_pushdown; mod push_down_filter; mod replace_with_order_preserving_variants; mod sanity_checker; mod test_utils; -mod partition_statistics; diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index 8440976cbc68b..f794a086764b8 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -17,99 +17,312 @@ #[cfg(test)] mod test { - use std::sync::Arc; - use datafusion::datasource::listing::{ListingOptions, ListingTable, ListingTableConfig}; - use datafusion::execution::SessionStateBuilder; + use arrow_schema::{DataType, Field, Schema, SortOptions}; + use datafusion::datasource::listing::ListingTable; use datafusion::prelude::SessionContext; use datafusion_catalog::TableProvider; - use datafusion_common::config::ConfigOptions; - use datafusion_datasource::ListingTableUrl; - use datafusion_datasource::source::DataSourceExec; - use datafusion_datasource_parquet::ParquetFormat; + use datafusion_common::stats::Precision; + use datafusion_common::{ScalarValue, Statistics}; use datafusion_execution::config::SessionConfig; - use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use datafusion_expr_common::operator::Operator; + use datafusion_physical_expr::expressions::{binary, lit, Column}; + use datafusion_physical_expr_common::physical_expr::PhysicalExpr; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; + use datafusion_physical_plan::filter::FilterExec; + use datafusion_physical_plan::projection::ProjectionExec; + use datafusion_physical_plan::sorts::sort::SortExec; + use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::ExecutionPlan; + use std::sync::Arc; - async fn generate_listing_table_with_statistics() -> Arc { - let testdata = datafusion::test_util::parquet_test_data(); - let filename = format!("{}/{}", testdata, "alltypes_tiny_pages.parquet"); - let table_path = ListingTableUrl::parse(filename).unwrap(); - let opt = ListingOptions::new(Arc::new(ParquetFormat::default())).with_collect_stat(true); - let rt = RuntimeEnvBuilder::new() - .build_arc() - .expect("could not build runtime environment"); - - let state = SessionContext::new_with_config_rt(SessionConfig::default(), rt).state(); - let schema = opt - .infer_schema( - &SessionStateBuilder::new().with_default_features().build(), - &table_path, - ) + async fn generate_listing_table_with_statistics( + target_partition: Option, + ) -> Arc { + // Delete the existing data directory if it exists + let data_dir = "./data/"; + let _ = std::fs::remove_dir_all(data_dir); + let mut session_config = SessionConfig::new().with_collect_statistics(true); + if let Some(partition) = target_partition { + session_config = session_config.with_target_partitions(partition); + } + let ctx = SessionContext::new_with_config(session_config); + // Create table with partition + let create_table_sql = "CREATE EXTERNAL TABLE t1 (id INT not null, date DATE) STORED AS PARQUET LOCATION './data/' PARTITIONED BY (date) WITH ORDER (id ASC);"; + ctx.sql(create_table_sql) + .await + .unwrap() + .collect() .await .unwrap(); - let config = ListingTableConfig::new(table_path.clone()) - .with_listing_options(opt.clone()) - .with_schema(schema); - let table = ListingTable::try_new(config).unwrap(); - let res= table.scan(&state, None, &[], None).await.unwrap(); - dbg!(&res.statistics().unwrap()); - dbg!(&res.statistics_by_partition().unwrap()); - let mut config = ConfigOptions::new(); - config.set("datafusion.optimizer.repartition_file_min_size", "10").unwrap(); - let res = res.repartitioned(5, &config).unwrap().unwrap(); - dbg!(&res.statistics_by_partition().unwrap()); - res + // Insert data into the table, will generate partition files with parquet format + let insert_data = "INSERT INTO t1 VALUES (4, '2025-03-01'), (3, '2025-3-02'), (2, '2025-03-03'), (1, '2025-03-04');"; + ctx.sql(insert_data).await.unwrap().collect().await.unwrap(); + let table = ctx.table_provider("t1").await.unwrap(); + let listing_table = table + .as_any() + .downcast_ref::() + .unwrap() + .clone(); + listing_table + .scan(&ctx.state(), None, &[], None) + .await + .unwrap() } - #[tokio::test] - async fn test_statistics_by_partition_of_data_source() -> datafusion_common::Result<()> { - generate_listing_table_with_statistics().await; - Ok(()) - } + fn check_unchanged_statistics(statistics: Vec) { + // Check the statistics of each partition + for stat in &statistics { + assert_eq!(stat.num_rows, Precision::Exact(1)); + // First column (id) should have non-null values + assert_eq!(stat.column_statistics[0].null_count, Precision::Exact(0)); + } - #[test] - fn test_statistics_by_partition_of_projection() -> datafusion_common::Result<()> { - Ok(()) + // Verify specific id values for each partition + assert_eq!( + statistics[0].column_statistics[0].max_value, + Precision::Exact(ScalarValue::Int32(Some(4))) + ); + assert_eq!( + statistics[1].column_statistics[0].max_value, + Precision::Exact(ScalarValue::Int32(Some(3))) + ); + assert_eq!( + statistics[2].column_statistics[0].max_value, + Precision::Exact(ScalarValue::Int32(Some(2))) + ); + assert_eq!( + statistics[3].column_statistics[0].max_value, + Precision::Exact(ScalarValue::Int32(Some(1))) + ); } - #[test] - fn test_statistics_by_partition_of_sort() -> datafusion_common::Result<()> { + #[tokio::test] + async fn test_statistics_by_partition_of_data_source() -> datafusion_common::Result<()> + { + let scan = generate_listing_table_with_statistics(None).await; + let statistics = scan.statistics_by_partition()?; + // Check the statistics of each partition + assert_eq!(statistics.len(), 4); + for stat in &statistics { + assert_eq!(stat.column_statistics.len(), 2); + assert_eq!(stat.total_byte_size, Precision::Exact(55)); + } + check_unchanged_statistics(statistics); Ok(()) } - #[test] - fn test_statistics_by_partition_of_filter() -> datafusion_common::Result<()> { + #[tokio::test] + async fn test_statistics_by_partition_of_projection() -> datafusion_common::Result<()> + { + let scan = generate_listing_table_with_statistics(None).await; + // Add projection execution plan + let exprs: Vec<(Arc, String)> = + vec![(Arc::new(Column::new("id", 0)), "id".to_string())]; + let projection = ProjectionExec::try_new(exprs, scan)?; + let statistics = projection.statistics_by_partition()?; + for stat in &statistics { + assert_eq!(stat.column_statistics.len(), 1); + assert_eq!(stat.total_byte_size, Precision::Exact(4)); + } + check_unchanged_statistics(statistics); Ok(()) } - #[test] - fn test_statistics_by_partition_of_aggregate() -> datafusion_common::Result<()> { + #[tokio::test] + async fn test_statistics_by_partition_of_sort() -> datafusion_common::Result<()> { + let scan = generate_listing_table_with_statistics(Some(2)).await; + // Add sort execution plan + let sort = SortExec::new( + LexOrdering::new(vec![PhysicalSortExpr { + expr: Arc::new(Column::new("id", 0)), + options: SortOptions { + descending: false, + nulls_first: false, + }, + }]), + scan, + ); + let mut sort_exec = Arc::new(sort.clone()); + let statistics = sort_exec.statistics_by_partition()?; + assert_eq!(statistics.len(), 1); + assert_eq!(statistics[0].num_rows, Precision::Exact(4)); + assert_eq!(statistics[0].column_statistics.len(), 2); + assert_eq!(statistics[0].total_byte_size, Precision::Exact(220)); + assert_eq!( + statistics[0].column_statistics[0].null_count, + Precision::Exact(0) + ); + assert_eq!( + statistics[0].column_statistics[0].max_value, + Precision::Exact(ScalarValue::Int32(Some(4))) + ); + assert_eq!( + statistics[0].column_statistics[0].min_value, + Precision::Exact(ScalarValue::Int32(Some(1))) + ); + sort_exec = Arc::new(sort.with_preserve_partitioning(true)); + let statistics = sort_exec.statistics_by_partition()?; + dbg!(&statistics); + assert_eq!(statistics.len(), 2); + assert_eq!(statistics[0].num_rows, Precision::Exact(2)); + assert_eq!(statistics[1].num_rows, Precision::Exact(2)); + assert_eq!(statistics[0].column_statistics.len(), 2); + assert_eq!(statistics[1].column_statistics.len(), 2); + assert_eq!(statistics[0].total_byte_size, Precision::Exact(110)); + assert_eq!(statistics[1].total_byte_size, Precision::Exact(110)); + assert_eq!( + statistics[0].column_statistics[0].null_count, + Precision::Exact(0) + ); + assert_eq!( + statistics[0].column_statistics[0].max_value, + Precision::Exact(ScalarValue::Int32(Some(4))) + ); + assert_eq!( + statistics[0].column_statistics[0].min_value, + Precision::Exact(ScalarValue::Int32(Some(3))) + ); + assert_eq!( + statistics[1].column_statistics[0].null_count, + Precision::Exact(0) + ); + assert_eq!( + statistics[1].column_statistics[0].max_value, + Precision::Exact(ScalarValue::Int32(Some(2))) + ); + assert_eq!( + statistics[1].column_statistics[0].min_value, + Precision::Exact(ScalarValue::Int32(Some(1))) + ); Ok(()) } - #[test] - fn test_statistic_by_partition_of_cross_join() -> datafusion_common::Result<()> { + #[tokio::test] + async fn test_statistics_by_partition_of_filter() -> datafusion_common::Result<()> { + let scan = generate_listing_table_with_statistics(None).await; + let schema = Schema::new(vec![Field::new("id", DataType::Int32, false)]); + let predicate = binary( + Arc::new(Column::new("id", 0)), + Operator::Lt, + lit(1i32), + &schema, + )?; + let filter: Arc = + Arc::new(FilterExec::try_new(predicate, scan)?); + let _full_statistics = filter.statistics()?; + // The full statistics is invalid, at least, we can improve the selectivity estimation of the filter + /* + Statistics { + num_rows: Inexact(0), + total_byte_size: Inexact(0), + column_statistics: [ + ColumnStatistics { + null_count: Exact(0), + max_value: Exact(NULL), + min_value: Exact(NULL), + sum_value: Exact(NULL), + distinct_count: Exact(0), + }, + ColumnStatistics { + null_count: Exact(0), + max_value: Exact(NULL), + min_value: Exact(NULL), + sum_value: Exact(NULL), + distinct_count: Exact(0), + }, + ], + } + */ + let statistics = filter.statistics_by_partition()?; + // Also the statistics of each partition is also invalid due to above + // But we can ensure the current behavior by tests + assert_eq!(statistics.len(), 4); + for stat in &statistics { + assert_eq!(stat.column_statistics.len(), 2); + assert_eq!(stat.total_byte_size, Precision::Inexact(0)); + assert_eq!(stat.num_rows, Precision::Inexact(0)); + assert_eq!(stat.column_statistics[0].null_count, Precision::Exact(0)); + assert_eq!( + stat.column_statistics[0].max_value, + Precision::Exact(ScalarValue::Null) + ); + assert_eq!( + stat.column_statistics[0].min_value, + Precision::Exact(ScalarValue::Null) + ); + } Ok(()) } - #[test] - fn test_statistic_by_partition_of_union() -> datafusion_common::Result<()> { - Ok(()) - } + #[tokio::test] + async fn test_statistic_by_partition_of_union() -> datafusion_common::Result<()> { + let scan = generate_listing_table_with_statistics(Some(2)).await; + let union_exec = Arc::new(UnionExec::new(vec![scan.clone(), scan])); + let statistics = union_exec.statistics_by_partition()?; + // Check that we have 4 partitions (2 from each scan) + assert_eq!(statistics.len(), 4); - #[test] - fn test_statistic_by_partition_of_smp() -> datafusion_common::Result<()> { - Ok(()) - } + // Verify first partition (from first scan) + assert_eq!(statistics[0].num_rows, Precision::Exact(2)); + assert_eq!(statistics[0].total_byte_size, Precision::Exact(110)); + assert_eq!(statistics[0].column_statistics.len(), 2); + assert_eq!( + statistics[0].column_statistics[0].null_count, + Precision::Exact(0) + ); + assert_eq!( + statistics[0].column_statistics[0].max_value, + Precision::Exact(ScalarValue::Int32(Some(4))) + ); + assert_eq!( + statistics[0].column_statistics[0].min_value, + Precision::Exact(ScalarValue::Int32(Some(3))) + ); - #[test] - fn test_statistic_by_partition_of_limit() -> datafusion_common::Result<()> { - Ok(()) - } + // Verify second partition (from first scan) + assert_eq!(statistics[1].num_rows, Precision::Exact(2)); + assert_eq!(statistics[1].total_byte_size, Precision::Exact(110)); + assert_eq!( + statistics[1].column_statistics[0].null_count, + Precision::Exact(0) + ); + assert_eq!( + statistics[1].column_statistics[0].max_value, + Precision::Exact(ScalarValue::Int32(Some(2))) + ); + assert_eq!( + statistics[1].column_statistics[0].min_value, + Precision::Exact(ScalarValue::Int32(Some(1))) + ); + + // Verify third partition (from second scan - same as first partition) + assert_eq!(statistics[2].num_rows, Precision::Exact(2)); + assert_eq!(statistics[2].total_byte_size, Precision::Exact(110)); + assert_eq!( + statistics[2].column_statistics[0].max_value, + Precision::Exact(ScalarValue::Int32(Some(4))) + ); + assert_eq!( + statistics[2].column_statistics[0].min_value, + Precision::Exact(ScalarValue::Int32(Some(3))) + ); + + // Verify fourth partition (from second scan - same as second partition) + assert_eq!(statistics[3].num_rows, Precision::Exact(2)); + assert_eq!(statistics[3].total_byte_size, Precision::Exact(110)); + assert_eq!( + statistics[3].column_statistics[0].max_value, + Precision::Exact(ScalarValue::Int32(Some(2))) + ); + assert_eq!( + statistics[3].column_statistics[0].min_value, + Precision::Exact(ScalarValue::Int32(Some(1))) + ); + + // Delete the existing data directory if it exists + let data_dir = "./data/"; + let _ = std::fs::remove_dir_all(data_dir); - #[test] - fn test_statistic_by_partition_of_coalesce() -> datafusion_common::Result<()> { Ok(()) } - -} \ No newline at end of file +} From d3b99b31e272f7d7ef5c55b9531654db524e274a Mon Sep 17 00:00:00 2001 From: xudong963 Date: Thu, 3 Apr 2025 16:07:43 +0800 Subject: [PATCH 08/26] save --- .../date=2025-03-01/j5fUeSDQo22oPyPU.parquet | Bin 0 -> 461 bytes .../date=2025-03-02/j5fUeSDQo22oPyPU.parquet | Bin 0 -> 461 bytes .../date=2025-03-03/j5fUeSDQo22oPyPU.parquet | Bin 0 -> 461 bytes .../date=2025-03-04/j5fUeSDQo22oPyPU.parquet | Bin 0 -> 461 bytes .../physical_optimizer/partition_statistics.rs | 13 +------------ 5 files changed, 1 insertion(+), 12 deletions(-) create mode 100644 datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet create mode 100644 datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet create mode 100644 datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet create mode 100644 datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet diff --git a/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet b/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet new file mode 100644 index 0000000000000000000000000000000000000000..ec164c6df7b5e0b73774c8e0e9f4ddb1cb84bea5 GIT binary patch literal 461 zcmZWm%Syvg5S?5~$RdkII+t9z3tbqxD6K{9La;E3XnkNTlqv`k+FFa+LaOFJT>1@e z{Re+aa3+bka299goHLJ05>vm874&fR3>Cr(K>f4z4Oi;`6#zhBQ#aVEqKmHWK)oV* zm}3xdS@cV#OrT8RlnIs50LJPr-f#iT8^?Z0qKlG{JzZmWoH^UF7C9yRn zj!OZKQ=*IxrgS0_XQKUFXUd$YCyb@v%xVg w71p1h?!4P;v1ul=(IS>9=fTVcys%b&IH0Cf3B{s11@e z{Re+aa3+bka299goHLJ05>vl}6|Caw87hPofaYiW8?M#?6aXNwsatH-(M8vGp;-~# z&(ZU^Ec&HV##5$n%7n^j0AqCzZ@2*Ft>apfri<)$Il7-bC8${30AJ0D1QxqfqkUQo5g_5$_4x9Bei8~WYB5O_odRUT`|eZNNi{>5;&xaog5ZHcWZ zaa;;;oDyYpFr^cbI1}yXI#cFEkua8iGqXdH1_E3S%Ko@q8I}B6{Cr3XibjIG?>xVg w71p1h?Am$(W11gqR-2eap literal 0 HcmV?d00001 diff --git a/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet b/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet new file mode 100644 index 0000000000000000000000000000000000000000..6398cc43a2f5dd659c2054929eb7a3600ccfb678 GIT binary patch literal 461 zcmZWm%Syvg5S?5~=%R~8I+t9z3tbqxD5*v5La;E3XnbHSlqv`k+FFa+LaOFJT>1@e z{Re+aa3+bka299goHLJ*8&ki76|Caw87hPofaYiW8?M#?8~{LIQ@7Zvql>QXLbD>e zpQGn-S@cV#jHgWDlnIs50LJPb-f#iTTgSB|O&8hia&$j=N>H)50lu1zK`Ud84Zw2S z4!CtAD%P~pbN%K7y`XA~?FH(8-=e=9Z0L6fL*Nk=RC%l+_x&36`xnFA;->%Mv?aEt z#BnLWaY~fY!IVx!;!L!k>r9ywMZ#G6&CCu(8VGPTDEs4bWmNKO@$(@mC>ja!zVrM} wR#<<2zLQAqA3Z)W?tyQ{$#Zh|vYajEYp;_u&)eN_Gwc{^`L_=FNn(D;A0B~M)Bpeg literal 0 HcmV?d00001 diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index f794a086764b8..d633c62899915 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -38,25 +38,19 @@ mod test { async fn generate_listing_table_with_statistics( target_partition: Option, ) -> Arc { - // Delete the existing data directory if it exists - let data_dir = "./data/"; - let _ = std::fs::remove_dir_all(data_dir); let mut session_config = SessionConfig::new().with_collect_statistics(true); if let Some(partition) = target_partition { session_config = session_config.with_target_partitions(partition); } let ctx = SessionContext::new_with_config(session_config); // Create table with partition - let create_table_sql = "CREATE EXTERNAL TABLE t1 (id INT not null, date DATE) STORED AS PARQUET LOCATION './data/' PARTITIONED BY (date) WITH ORDER (id ASC);"; + let create_table_sql = "CREATE EXTERNAL TABLE t1 (id INT not null, date DATE) STORED AS PARQUET LOCATION './tests/data/test_statistics_per_partition' PARTITIONED BY (date) WITH ORDER (id ASC);"; ctx.sql(create_table_sql) .await .unwrap() .collect() .await .unwrap(); - // Insert data into the table, will generate partition files with parquet format - let insert_data = "INSERT INTO t1 VALUES (4, '2025-03-01'), (3, '2025-3-02'), (2, '2025-03-03'), (1, '2025-03-04');"; - ctx.sql(insert_data).await.unwrap().collect().await.unwrap(); let table = ctx.table_provider("t1").await.unwrap(); let listing_table = table .as_any() @@ -162,7 +156,6 @@ mod test { ); sort_exec = Arc::new(sort.with_preserve_partitioning(true)); let statistics = sort_exec.statistics_by_partition()?; - dbg!(&statistics); assert_eq!(statistics.len(), 2); assert_eq!(statistics[0].num_rows, Precision::Exact(2)); assert_eq!(statistics[1].num_rows, Precision::Exact(2)); @@ -319,10 +312,6 @@ mod test { Precision::Exact(ScalarValue::Int32(Some(1))) ); - // Delete the existing data directory if it exists - let data_dir = "./data/"; - let _ = std::fs::remove_dir_all(data_dir); - Ok(()) } } From 972a2bb1a1e2f96554a9a9fa98a124a1cffc40d6 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Thu, 3 Apr 2025 16:20:54 +0800 Subject: [PATCH 09/26] update --- .../partition_statistics.rs | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index d633c62899915..b98c499e1b240 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -66,7 +66,7 @@ mod test { fn check_unchanged_statistics(statistics: Vec) { // Check the statistics of each partition for stat in &statistics { - assert_eq!(stat.num_rows, Precision::Exact(1)); + assert_eq!(stat.num_rows, Precision::Exact(2)); // First column (id) should have non-null values assert_eq!(stat.column_statistics[0].null_count, Precision::Exact(0)); } @@ -77,15 +77,15 @@ mod test { Precision::Exact(ScalarValue::Int32(Some(4))) ); assert_eq!( - statistics[1].column_statistics[0].max_value, + statistics[0].column_statistics[0].min_value, Precision::Exact(ScalarValue::Int32(Some(3))) ); assert_eq!( - statistics[2].column_statistics[0].max_value, + statistics[1].column_statistics[0].max_value, Precision::Exact(ScalarValue::Int32(Some(2))) ); assert_eq!( - statistics[3].column_statistics[0].max_value, + statistics[1].column_statistics[0].min_value, Precision::Exact(ScalarValue::Int32(Some(1))) ); } @@ -93,13 +93,13 @@ mod test { #[tokio::test] async fn test_statistics_by_partition_of_data_source() -> datafusion_common::Result<()> { - let scan = generate_listing_table_with_statistics(None).await; + let scan = generate_listing_table_with_statistics(Some(2)).await; let statistics = scan.statistics_by_partition()?; // Check the statistics of each partition - assert_eq!(statistics.len(), 4); + assert_eq!(statistics.len(), 2); for stat in &statistics { assert_eq!(stat.column_statistics.len(), 2); - assert_eq!(stat.total_byte_size, Precision::Exact(55)); + assert_eq!(stat.total_byte_size, Precision::Exact(110)); } check_unchanged_statistics(statistics); Ok(()) @@ -108,7 +108,7 @@ mod test { #[tokio::test] async fn test_statistics_by_partition_of_projection() -> datafusion_common::Result<()> { - let scan = generate_listing_table_with_statistics(None).await; + let scan = generate_listing_table_with_statistics(Some(2)).await; // Add projection execution plan let exprs: Vec<(Arc, String)> = vec![(Arc::new(Column::new("id", 0)), "id".to_string())]; @@ -116,7 +116,7 @@ mod test { let statistics = projection.statistics_by_partition()?; for stat in &statistics { assert_eq!(stat.column_statistics.len(), 1); - assert_eq!(stat.total_byte_size, Precision::Exact(4)); + assert_eq!(stat.total_byte_size, Precision::Exact(8)); } check_unchanged_statistics(statistics); Ok(()) @@ -192,7 +192,7 @@ mod test { #[tokio::test] async fn test_statistics_by_partition_of_filter() -> datafusion_common::Result<()> { - let scan = generate_listing_table_with_statistics(None).await; + let scan = generate_listing_table_with_statistics(Some(2)).await; let schema = Schema::new(vec![Field::new("id", DataType::Int32, false)]); let predicate = binary( Arc::new(Column::new("id", 0)), @@ -229,7 +229,7 @@ mod test { let statistics = filter.statistics_by_partition()?; // Also the statistics of each partition is also invalid due to above // But we can ensure the current behavior by tests - assert_eq!(statistics.len(), 4); + assert_eq!(statistics.len(), 2); for stat in &statistics { assert_eq!(stat.column_statistics.len(), 2); assert_eq!(stat.total_byte_size, Precision::Inexact(0)); From 7ce04d518b4da3d7cc7814e3b69b0a13fee324f0 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Thu, 3 Apr 2025 22:54:23 +0800 Subject: [PATCH 10/26] add PartitionedStatistics structure --- datafusion/physical-plan/src/statistics.rs | 45 ++++++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/datafusion/physical-plan/src/statistics.rs b/datafusion/physical-plan/src/statistics.rs index 3d70d907219d4..3e5c1006a84a7 100644 --- a/datafusion/physical-plan/src/statistics.rs +++ b/datafusion/physical-plan/src/statistics.rs @@ -22,6 +22,51 @@ use datafusion_common::stats::Precision; use datafusion_common::{ColumnStatistics, ScalarValue, Statistics}; use std::mem; +/// Represents statistics data grouped by partition. +/// +/// This structure maintains a collection of statistics, one for each partition +/// of a distributed dataset, allowing access to statistics by partition index. +#[derive(Debug, Clone)] +pub struct PartitionedStatistics { + inner: Vec, +} + +impl PartitionedStatistics { + /// Creates a new PartitionedStatistics instance from a vector of Statistics. + pub fn new(statistics: Vec) -> Self { + Self { inner: statistics } + } + + /// Returns the number of partitions. + pub fn len(&self) -> usize { + self.inner.len() + } + + /// Returns true if there are no partitions. + pub fn is_empty(&self) -> bool { + self.inner.is_empty() + } + + /// Returns the statistics for the specified partition. + /// + /// # Panics + /// + /// Panics if `partition_idx` is out of bounds. + pub fn statistics(&self, partition_idx: usize) -> &Statistics { + &self.inner[partition_idx] + } + + /// Returns the statistics for the specified partition, or None if the index is out of bounds. + pub fn get_statistics(&self, partition_idx: usize) -> Option<&Statistics> { + self.inner.get(partition_idx) + } + + /// Returns an iterator over the statistics. + pub fn iter(&self) -> impl Iterator { + self.inner.iter() + } +} + /// Generic function to compute statistics across multiple items that have statistics pub fn compute_summary_statistics( items: I, From c3f92c69786239bda4ea94ff2f8a7e11b1db10cb Mon Sep 17 00:00:00 2001 From: xudong963 Date: Thu, 3 Apr 2025 23:08:05 +0800 Subject: [PATCH 11/26] use Arc --- datafusion/physical-plan/src/statistics.rs | 27 ++++------------------ 1 file changed, 5 insertions(+), 22 deletions(-) diff --git a/datafusion/physical-plan/src/statistics.rs b/datafusion/physical-plan/src/statistics.rs index 3e5c1006a84a7..0d8f15f0947df 100644 --- a/datafusion/physical-plan/src/statistics.rs +++ b/datafusion/physical-plan/src/statistics.rs @@ -21,6 +21,7 @@ use datafusion_common::stats::Precision; use datafusion_common::{ColumnStatistics, ScalarValue, Statistics}; use std::mem; +use std::sync::Arc; /// Represents statistics data grouped by partition. /// @@ -28,42 +29,24 @@ use std::mem; /// of a distributed dataset, allowing access to statistics by partition index. #[derive(Debug, Clone)] pub struct PartitionedStatistics { - inner: Vec, + inner: Vec>, } impl PartitionedStatistics { - /// Creates a new PartitionedStatistics instance from a vector of Statistics. - pub fn new(statistics: Vec) -> Self { + pub fn new(statistics: Vec>) -> Self { Self { inner: statistics } } - /// Returns the number of partitions. - pub fn len(&self) -> usize { - self.inner.len() - } - - /// Returns true if there are no partitions. - pub fn is_empty(&self) -> bool { - self.inner.is_empty() - } - - /// Returns the statistics for the specified partition. - /// - /// # Panics - /// - /// Panics if `partition_idx` is out of bounds. pub fn statistics(&self, partition_idx: usize) -> &Statistics { &self.inner[partition_idx] } - /// Returns the statistics for the specified partition, or None if the index is out of bounds. pub fn get_statistics(&self, partition_idx: usize) -> Option<&Statistics> { - self.inner.get(partition_idx) + self.inner.get(partition_idx).map(|arc| arc.as_ref()) } - /// Returns an iterator over the statistics. pub fn iter(&self) -> impl Iterator { - self.inner.iter() + self.inner.iter().map(|arc| arc.as_ref()) } } From 3b88d8a15ac670e05a75016aeae5cee8523f6a90 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Fri, 4 Apr 2025 00:22:49 +0800 Subject: [PATCH 12/26] refine tests --- .../partition_statistics.rs | 271 +++++++----------- 1 file changed, 99 insertions(+), 172 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index b98c499e1b240..3da9488a9dfe8 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -22,7 +22,7 @@ mod test { use datafusion::prelude::SessionContext; use datafusion_catalog::TableProvider; use datafusion_common::stats::Precision; - use datafusion_common::{ScalarValue, Statistics}; + use datafusion_common::{ColumnStatistics, ScalarValue, Statistics}; use datafusion_execution::config::SessionConfig; use datafusion_expr_common::operator::Operator; use datafusion_physical_expr::expressions::{binary, lit, Column}; @@ -35,6 +35,18 @@ mod test { use datafusion_physical_plan::ExecutionPlan; use std::sync::Arc; + /// Creates a test table with statistics from the test data directory. + /// + /// This function: + /// - Creates an external table from './tests/data/test_statistics_per_partition' + /// - If we set the `target_partition` to `2, the data contains 2 partitions, each with 2 rows + /// - Each partition has an "id" column (INT) with the following values: + /// - First partition: [3, 4] + /// - Second partition: [1, 2] + /// - Each row is 110 bytes in size + /// + /// @param target_partition Optional parameter to set the target partitions + /// @return ExecutionPlan representing the scan of the table with statistics async fn generate_listing_table_with_statistics( target_partition: Option, ) -> Arc { @@ -63,31 +75,37 @@ mod test { .unwrap() } - fn check_unchanged_statistics(statistics: Vec) { - // Check the statistics of each partition - for stat in &statistics { - assert_eq!(stat.num_rows, Precision::Exact(2)); - // First column (id) should have non-null values - assert_eq!(stat.column_statistics[0].null_count, Precision::Exact(0)); + /// Helper function to create expected statistics for a partition with Int32 column + fn create_partition_statistics( + num_rows: usize, + total_byte_size: usize, + min_value: i32, + max_value: i32, + include_date_column: bool, + ) -> Statistics { + let mut column_stats = vec![ColumnStatistics { + null_count: Precision::Exact(0), + max_value: Precision::Exact(ScalarValue::Int32(Some(max_value))), + min_value: Precision::Exact(ScalarValue::Int32(Some(min_value))), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + }]; + + if include_date_column { + column_stats.push(ColumnStatistics { + null_count: Precision::Absent, + max_value: Precision::Absent, + min_value: Precision::Absent, + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + }); } - // Verify specific id values for each partition - assert_eq!( - statistics[0].column_statistics[0].max_value, - Precision::Exact(ScalarValue::Int32(Some(4))) - ); - assert_eq!( - statistics[0].column_statistics[0].min_value, - Precision::Exact(ScalarValue::Int32(Some(3))) - ); - assert_eq!( - statistics[1].column_statistics[0].max_value, - Precision::Exact(ScalarValue::Int32(Some(2))) - ); - assert_eq!( - statistics[1].column_statistics[0].min_value, - Precision::Exact(ScalarValue::Int32(Some(1))) - ); + Statistics { + num_rows: Precision::Exact(num_rows), + total_byte_size: Precision::Exact(total_byte_size), + column_statistics: column_stats, + } } #[tokio::test] @@ -95,13 +113,14 @@ mod test { { let scan = generate_listing_table_with_statistics(Some(2)).await; let statistics = scan.statistics_by_partition()?; + let expected_statistic_partition_1 = + create_partition_statistics(2, 110, 3, 4, true); + let expected_statistic_partition_2 = + create_partition_statistics(2, 110, 1, 2, true); // Check the statistics of each partition assert_eq!(statistics.len(), 2); - for stat in &statistics { - assert_eq!(stat.column_statistics.len(), 2); - assert_eq!(stat.total_byte_size, Precision::Exact(110)); - } - check_unchanged_statistics(statistics); + assert_eq!(statistics[0], expected_statistic_partition_1); + assert_eq!(statistics[1], expected_statistic_partition_2); Ok(()) } @@ -114,11 +133,14 @@ mod test { vec![(Arc::new(Column::new("id", 0)), "id".to_string())]; let projection = ProjectionExec::try_new(exprs, scan)?; let statistics = projection.statistics_by_partition()?; - for stat in &statistics { - assert_eq!(stat.column_statistics.len(), 1); - assert_eq!(stat.total_byte_size, Precision::Exact(8)); - } - check_unchanged_statistics(statistics); + let expected_statistic_partition_1 = + create_partition_statistics(2, 8, 3, 4, false); + let expected_statistic_partition_2 = + create_partition_statistics(2, 8, 1, 2, false); + // Check the statistics of each partition + assert_eq!(statistics.len(), 2); + assert_eq!(statistics[0], expected_statistic_partition_1); + assert_eq!(statistics[1], expected_statistic_partition_2); Ok(()) } @@ -138,55 +160,20 @@ mod test { ); let mut sort_exec = Arc::new(sort.clone()); let statistics = sort_exec.statistics_by_partition()?; + let expected_statistic_partition = + create_partition_statistics(4, 220, 1, 4, true); assert_eq!(statistics.len(), 1); - assert_eq!(statistics[0].num_rows, Precision::Exact(4)); - assert_eq!(statistics[0].column_statistics.len(), 2); - assert_eq!(statistics[0].total_byte_size, Precision::Exact(220)); - assert_eq!( - statistics[0].column_statistics[0].null_count, - Precision::Exact(0) - ); - assert_eq!( - statistics[0].column_statistics[0].max_value, - Precision::Exact(ScalarValue::Int32(Some(4))) - ); - assert_eq!( - statistics[0].column_statistics[0].min_value, - Precision::Exact(ScalarValue::Int32(Some(1))) - ); + assert_eq!(statistics[0], expected_statistic_partition); + sort_exec = Arc::new(sort.with_preserve_partitioning(true)); + let expected_statistic_partition_1 = + create_partition_statistics(2, 110, 3, 4, true); + let expected_statistic_partition_2 = + create_partition_statistics(2, 110, 1, 2, true); let statistics = sort_exec.statistics_by_partition()?; assert_eq!(statistics.len(), 2); - assert_eq!(statistics[0].num_rows, Precision::Exact(2)); - assert_eq!(statistics[1].num_rows, Precision::Exact(2)); - assert_eq!(statistics[0].column_statistics.len(), 2); - assert_eq!(statistics[1].column_statistics.len(), 2); - assert_eq!(statistics[0].total_byte_size, Precision::Exact(110)); - assert_eq!(statistics[1].total_byte_size, Precision::Exact(110)); - assert_eq!( - statistics[0].column_statistics[0].null_count, - Precision::Exact(0) - ); - assert_eq!( - statistics[0].column_statistics[0].max_value, - Precision::Exact(ScalarValue::Int32(Some(4))) - ); - assert_eq!( - statistics[0].column_statistics[0].min_value, - Precision::Exact(ScalarValue::Int32(Some(3))) - ); - assert_eq!( - statistics[1].column_statistics[0].null_count, - Precision::Exact(0) - ); - assert_eq!( - statistics[1].column_statistics[0].max_value, - Precision::Exact(ScalarValue::Int32(Some(2))) - ); - assert_eq!( - statistics[1].column_statistics[0].min_value, - Precision::Exact(ScalarValue::Int32(Some(1))) - ); + assert_eq!(statistics[0], expected_statistic_partition_1); + assert_eq!(statistics[1], expected_statistic_partition_2); Ok(()) } @@ -202,48 +189,33 @@ mod test { )?; let filter: Arc = Arc::new(FilterExec::try_new(predicate, scan)?); - let _full_statistics = filter.statistics()?; - // The full statistics is invalid, at least, we can improve the selectivity estimation of the filter - /* - Statistics { - num_rows: Inexact(0), - total_byte_size: Inexact(0), - column_statistics: [ - ColumnStatistics { - null_count: Exact(0), - max_value: Exact(NULL), - min_value: Exact(NULL), - sum_value: Exact(NULL), - distinct_count: Exact(0), - }, - ColumnStatistics { - null_count: Exact(0), - max_value: Exact(NULL), - min_value: Exact(NULL), - sum_value: Exact(NULL), - distinct_count: Exact(0), - }, - ], - } - */ + let full_statistics = filter.statistics()?; + let expected_full_statistic = Statistics { + num_rows: Precision::Inexact(0), + total_byte_size: Precision::Inexact(0), + column_statistics: vec![ + ColumnStatistics { + null_count: Precision::Exact(0), + max_value: Precision::Exact(ScalarValue::Null), + min_value: Precision::Exact(ScalarValue::Null), + sum_value: Precision::Exact(ScalarValue::Null), + distinct_count: Precision::Exact(0), + }, + ColumnStatistics { + null_count: Precision::Exact(0), + max_value: Precision::Exact(ScalarValue::Null), + min_value: Precision::Exact(ScalarValue::Null), + sum_value: Precision::Exact(ScalarValue::Null), + distinct_count: Precision::Exact(0), + }, + ], + }; + assert_eq!(full_statistics, expected_full_statistic); + let statistics = filter.statistics_by_partition()?; - // Also the statistics of each partition is also invalid due to above - // But we can ensure the current behavior by tests assert_eq!(statistics.len(), 2); - for stat in &statistics { - assert_eq!(stat.column_statistics.len(), 2); - assert_eq!(stat.total_byte_size, Precision::Inexact(0)); - assert_eq!(stat.num_rows, Precision::Inexact(0)); - assert_eq!(stat.column_statistics[0].null_count, Precision::Exact(0)); - assert_eq!( - stat.column_statistics[0].max_value, - Precision::Exact(ScalarValue::Null) - ); - assert_eq!( - stat.column_statistics[0].min_value, - Precision::Exact(ScalarValue::Null) - ); - } + assert_eq!(statistics[0], expected_full_statistic); + assert_eq!(statistics[1], expected_full_statistic); Ok(()) } @@ -254,63 +226,18 @@ mod test { let statistics = union_exec.statistics_by_partition()?; // Check that we have 4 partitions (2 from each scan) assert_eq!(statistics.len(), 4); - + let expected_statistic_partition_1 = + create_partition_statistics(2, 110, 3, 4, true); + let expected_statistic_partition_2 = + create_partition_statistics(2, 110, 1, 2, true); // Verify first partition (from first scan) - assert_eq!(statistics[0].num_rows, Precision::Exact(2)); - assert_eq!(statistics[0].total_byte_size, Precision::Exact(110)); - assert_eq!(statistics[0].column_statistics.len(), 2); - assert_eq!( - statistics[0].column_statistics[0].null_count, - Precision::Exact(0) - ); - assert_eq!( - statistics[0].column_statistics[0].max_value, - Precision::Exact(ScalarValue::Int32(Some(4))) - ); - assert_eq!( - statistics[0].column_statistics[0].min_value, - Precision::Exact(ScalarValue::Int32(Some(3))) - ); - + assert_eq!(statistics[0], expected_statistic_partition_1); // Verify second partition (from first scan) - assert_eq!(statistics[1].num_rows, Precision::Exact(2)); - assert_eq!(statistics[1].total_byte_size, Precision::Exact(110)); - assert_eq!( - statistics[1].column_statistics[0].null_count, - Precision::Exact(0) - ); - assert_eq!( - statistics[1].column_statistics[0].max_value, - Precision::Exact(ScalarValue::Int32(Some(2))) - ); - assert_eq!( - statistics[1].column_statistics[0].min_value, - Precision::Exact(ScalarValue::Int32(Some(1))) - ); - + assert_eq!(statistics[1], expected_statistic_partition_2); // Verify third partition (from second scan - same as first partition) - assert_eq!(statistics[2].num_rows, Precision::Exact(2)); - assert_eq!(statistics[2].total_byte_size, Precision::Exact(110)); - assert_eq!( - statistics[2].column_statistics[0].max_value, - Precision::Exact(ScalarValue::Int32(Some(4))) - ); - assert_eq!( - statistics[2].column_statistics[0].min_value, - Precision::Exact(ScalarValue::Int32(Some(3))) - ); - + assert_eq!(statistics[2], expected_statistic_partition_1); // Verify fourth partition (from second scan - same as second partition) - assert_eq!(statistics[3].num_rows, Precision::Exact(2)); - assert_eq!(statistics[3].total_byte_size, Precision::Exact(110)); - assert_eq!( - statistics[3].column_statistics[0].max_value, - Precision::Exact(ScalarValue::Int32(Some(2))) - ); - assert_eq!( - statistics[3].column_statistics[0].min_value, - Precision::Exact(ScalarValue::Int32(Some(1))) - ); + assert_eq!(statistics[3], expected_statistic_partition_2); Ok(()) } From fc4a51eb3431a787f696730d2a42dac2c5a40895 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 7 Apr 2025 16:50:41 +0800 Subject: [PATCH 13/26] save --- .../src/datasource/physical_plan/arrow_file.rs | 2 +- .../physical_optimizer/partition_statistics.rs | 14 +++++++------- datafusion/datasource-avro/src/source.rs | 2 +- datafusion/datasource-csv/src/source.rs | 2 +- datafusion/physical-plan/src/coalesce_batches.rs | 8 ++++++-- .../physical-plan/src/coalesce_partitions.rs | 2 +- datafusion/physical-plan/src/execution_plan.rs | 9 +++++---- datafusion/physical-plan/src/filter.rs | 2 +- datafusion/physical-plan/src/joins/cross_join.rs | 2 +- datafusion/physical-plan/src/limit.rs | 4 ++-- datafusion/physical-plan/src/projection.rs | 2 +- datafusion/physical-plan/src/sorts/sort.rs | 2 +- .../src/sorts/sort_preserving_merge.rs | 2 +- datafusion/physical-plan/src/union.rs | 2 +- .../src/windows/bounded_window_agg_exec.rs | 2 +- 15 files changed, 31 insertions(+), 26 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index ba23cf4095398..72723b8f20fb2 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -191,7 +191,7 @@ impl ExecutionPlan for ArrowExec { self.inner.statistics() } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { self.inner.statistics_by_partition() } diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index 3da9488a9dfe8..4292b9b3bb753 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -39,7 +39,7 @@ mod test { /// /// This function: /// - Creates an external table from './tests/data/test_statistics_per_partition' - /// - If we set the `target_partition` to `2, the data contains 2 partitions, each with 2 rows + /// - If we set the `target_partition` to 2, the data contains 2 partitions, each with 2 rows /// - Each partition has an "id" column (INT) with the following values: /// - First partition: [3, 4] /// - Second partition: [1, 2] @@ -47,7 +47,7 @@ mod test { /// /// @param target_partition Optional parameter to set the target partitions /// @return ExecutionPlan representing the scan of the table with statistics - async fn generate_listing_table_with_statistics( + async fn create_scan_exec_with_statistics( target_partition: Option, ) -> Arc { let mut session_config = SessionConfig::new().with_collect_statistics(true); @@ -111,7 +111,7 @@ mod test { #[tokio::test] async fn test_statistics_by_partition_of_data_source() -> datafusion_common::Result<()> { - let scan = generate_listing_table_with_statistics(Some(2)).await; + let scan = create_scan_exec_with_statistics(Some(2)).await; let statistics = scan.statistics_by_partition()?; let expected_statistic_partition_1 = create_partition_statistics(2, 110, 3, 4, true); @@ -127,7 +127,7 @@ mod test { #[tokio::test] async fn test_statistics_by_partition_of_projection() -> datafusion_common::Result<()> { - let scan = generate_listing_table_with_statistics(Some(2)).await; + let scan = create_scan_exec_with_statistics(Some(2)).await; // Add projection execution plan let exprs: Vec<(Arc, String)> = vec![(Arc::new(Column::new("id", 0)), "id".to_string())]; @@ -146,7 +146,7 @@ mod test { #[tokio::test] async fn test_statistics_by_partition_of_sort() -> datafusion_common::Result<()> { - let scan = generate_listing_table_with_statistics(Some(2)).await; + let scan = create_scan_exec_with_statistics(Some(2)).await; // Add sort execution plan let sort = SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { @@ -179,7 +179,7 @@ mod test { #[tokio::test] async fn test_statistics_by_partition_of_filter() -> datafusion_common::Result<()> { - let scan = generate_listing_table_with_statistics(Some(2)).await; + let scan = create_scan_exec_with_statistics(Some(2)).await; let schema = Schema::new(vec![Field::new("id", DataType::Int32, false)]); let predicate = binary( Arc::new(Column::new("id", 0)), @@ -221,7 +221,7 @@ mod test { #[tokio::test] async fn test_statistic_by_partition_of_union() -> datafusion_common::Result<()> { - let scan = generate_listing_table_with_statistics(Some(2)).await; + let scan = create_scan_exec_with_statistics(Some(2)).await; let union_exec = Arc::new(UnionExec::new(vec![scan.clone(), scan])); let statistics = union_exec.statistics_by_partition()?; // Check that we have 4 partitions (2 from each scan) diff --git a/datafusion/datasource-avro/src/source.rs b/datafusion/datasource-avro/src/source.rs index 5c662511929c7..4a81798619c34 100644 --- a/datafusion/datasource-avro/src/source.rs +++ b/datafusion/datasource-avro/src/source.rs @@ -141,7 +141,7 @@ impl ExecutionPlan for AvroExec { self.inner.statistics() } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { self.inner.statistics_by_partition() } diff --git a/datafusion/datasource-csv/src/source.rs b/datafusion/datasource-csv/src/source.rs index 22fbb288c0396..5ca086d03af36 100644 --- a/datafusion/datasource-csv/src/source.rs +++ b/datafusion/datasource-csv/src/source.rs @@ -381,7 +381,7 @@ impl ExecutionPlan for CsvExec { self.inner.statistics() } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { self.inner.statistics_by_partition() } diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 314e5548b36a9..2bdd1e58c2d1a 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -199,8 +199,12 @@ impl ExecutionPlan for CoalesceBatchesExec { Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } - fn statistics_by_partition(&self) -> Result> { - Ok(vec![self.statistics()?]) + fn statistics_by_partition(&self) -> Result { + self.input + .statistics_by_partition()? + .into_iter() + .map(|stat| Statistics::with_fetch(stat, self.schema(), self.fetch, 0, 1)) + .collect() } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 1de838e830c23..7835b97244737 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -199,7 +199,7 @@ impl ExecutionPlan for CoalescePartitionsExec { Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { Ok(vec![self.statistics()?]) } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index ab69ac9af32a2..49aa46b59b577 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -55,6 +55,7 @@ use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::stream::{StreamExt, TryStreamExt}; +use crate::statistics::PartitionedStatistics; /// Represent nodes in the DataFusion Physical Plan. /// @@ -433,11 +434,11 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Returns statistics for each partition of this `ExecutionPlan` node. /// If statistics are not available, returns an array of /// [`Statistics::new_unknown`] for each partition. - fn statistics_by_partition(&self) -> Result> { - Ok(vec![ - Statistics::new_unknown(&self.schema()); + fn statistics_by_partition(&self) -> Result { + Ok(PartitionedStatistics::new(vec![ + Arc::new(Statistics::new_unknown(&self.schema())); self.properties().partitioning.partition_count() - ]) + ])) } /// Returns `true` if a limit can be safely pushed down through this diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 8378974be5d9b..a0c49347f0fb3 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -413,7 +413,7 @@ impl ExecutionPlan for FilterExec { Ok(stats.project(self.projection.as_ref())) } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { self.input .statistics_by_partition()? .into_iter() diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 3ad5276a547c6..a7c2f20c253b5 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -344,7 +344,7 @@ impl ExecutionPlan for CrossJoinExec { )) } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { let left_stats = self.left.statistics_by_partition()?; let right_stats = self.right.statistics_by_partition()?; diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 436b937f7086b..00b4a68499839 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -202,7 +202,7 @@ impl ExecutionPlan for GlobalLimitExec { ) } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { Ok(vec![self.statistics()?]) } @@ -347,7 +347,7 @@ impl ExecutionPlan for LocalLimitExec { ) } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { self.input .statistics_by_partition()? .into_iter() diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 629d9a2802af7..789e80dea672f 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -251,7 +251,7 @@ impl ExecutionPlan for ProjectionExec { )) } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { Ok(self .input .statistics_by_partition()? diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index e6f52496d3110..4d35f98b081c6 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1300,7 +1300,7 @@ impl ExecutionPlan for SortExec { Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { if !self.preserve_partitioning() { return Ok(vec![self.statistics()?]); } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index a1cfb3233751f..b4825a1f59247 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -346,7 +346,7 @@ impl ExecutionPlan for SortPreservingMergeExec { self.input.statistics() } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { Ok(vec![self.statistics()?]) } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 7225b1eee4dc6..45533c4cdfd81 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -270,7 +270,7 @@ impl ExecutionPlan for UnionExec { .unwrap_or_else(|| Statistics::new_unknown(&self.schema()))) } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { Ok(self .inputs .iter() 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 139b1e3414a7e..3653ee949c744 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -364,7 +364,7 @@ impl ExecutionPlan for BoundedWindowAggExec { self.statistics_helper(input_stat) } - fn statistics_by_partition(&self) -> Result> { + fn statistics_by_partition(&self) -> Result { self.input .statistics_by_partition()? .into_iter() From eac021cfc902e1b4c86e94057beb1c6c91d1fe55 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 7 Apr 2025 16:52:28 +0800 Subject: [PATCH 14/26] resolve conflicts --- datafusion/datasource/src/statistics.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/datasource/src/statistics.rs b/datafusion/datasource/src/statistics.rs index 5b44860091256..f26099a815995 100644 --- a/datafusion/datasource/src/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -36,7 +36,7 @@ use datafusion_common::stats::Precision; use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; use datafusion_physical_expr::{expressions::Column, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::statistics::{ +pub(crate) use datafusion_physical_plan::statistics::{ add_row_stats, compute_summary_statistics, set_max_if_greater, set_min_if_lesser, }; use datafusion_physical_plan::{ColumnStatistics, Statistics}; From 4decb735536ab557be3f009a24ef9bb9c8a6500a Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 7 Apr 2025 19:03:46 +0800 Subject: [PATCH 15/26] use PartitionedStatistics --- .../datasource/physical_plan/arrow_file.rs | 1 + datafusion/datasource-avro/src/source.rs | 1 + datafusion/datasource-csv/src/source.rs | 1 + datafusion/datasource/src/file_groups.rs | 7 ++++- datafusion/datasource/src/source.rs | 8 +++--- .../physical-plan/src/coalesce_batches.rs | 22 +++++++++++---- .../physical-plan/src/coalesce_partitions.rs | 5 +++- .../physical-plan/src/execution_plan.rs | 10 ++++--- datafusion/physical-plan/src/filter.rs | 21 +++++++++------ .../physical-plan/src/joins/cross_join.rs | 27 +++++++++++-------- datafusion/physical-plan/src/limit.rs | 27 ++++++++++++++----- datafusion/physical-plan/src/projection.rs | 26 +++++++++--------- datafusion/physical-plan/src/sorts/sort.rs | 27 ++++++++++++++----- .../src/sorts/sort_preserving_merge.rs | 5 +++- datafusion/physical-plan/src/statistics.rs | 4 +++ datafusion/physical-plan/src/union.rs | 17 +++++++----- .../src/windows/bounded_window_agg_exec.rs | 15 +++++++---- 17 files changed, 155 insertions(+), 69 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 72723b8f20fb2..5a237c29372f2 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -42,6 +42,7 @@ use datafusion_physical_plan::{ }; use datafusion_datasource::file_groups::FileGroup; +use datafusion_physical_plan::statistics::PartitionedStatistics; use futures::StreamExt; use itertools::Itertools; use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore}; diff --git a/datafusion/datasource-avro/src/source.rs b/datafusion/datasource-avro/src/source.rs index 4a81798619c34..1a0feea3f1535 100644 --- a/datafusion/datasource-avro/src/source.rs +++ b/datafusion/datasource-avro/src/source.rs @@ -40,6 +40,7 @@ use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, }; +use datafusion_physical_plan::statistics::PartitionedStatistics; use object_store::ObjectStore; /// Execution plan for scanning Avro data source diff --git a/datafusion/datasource-csv/src/source.rs b/datafusion/datasource-csv/src/source.rs index 5ca086d03af36..28aad5e43f147 100644 --- a/datafusion/datasource-csv/src/source.rs +++ b/datafusion/datasource-csv/src/source.rs @@ -51,6 +51,7 @@ use datafusion_physical_plan::{ use crate::file_format::CsvDecoder; use datafusion_datasource::file_groups::FileGroup; +use datafusion_physical_plan::statistics::PartitionedStatistics; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; use object_store::{GetOptions, GetResultPayload, ObjectStore}; diff --git a/datafusion/datasource/src/file_groups.rs b/datafusion/datasource/src/file_groups.rs index 929787e436c8d..ebf53d915777f 100644 --- a/datafusion/datasource/src/file_groups.rs +++ b/datafusion/datasource/src/file_groups.rs @@ -421,7 +421,12 @@ impl FileGroup { } /// Get the statistics for this group - pub fn statistics(&self) -> Option<&Statistics> { + pub fn statistics(&self) -> &Option> { + &self.statistics + } + + /// Get the statistics for this group + pub fn statistics_ref(&self) -> Option<&Statistics> { self.statistics.as_deref() } diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index ad6b8d3a719b6..3958575949f5b 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -201,9 +201,11 @@ impl ExecutionPlan for DataSourceExec { self.data_source.statistics() } - fn statistics_by_partition(&self) -> datafusion_common::Result> { + fn statistics_by_partition( + &self, + ) -> datafusion_common::Result { let mut statistics = vec![ - Statistics::new_unknown(&self.schema()); + Arc::new(Statistics::new_unknown(&self.schema())); self.properties().partitioning.partition_count() ]; if let Some(file_config) = @@ -215,7 +217,7 @@ impl ExecutionPlan for DataSourceExec { } } } - Ok(statistics) + Ok(PartitionedStatistics::new(statistics)) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 2bdd1e58c2d1a..3cebd53921a21 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -200,11 +200,23 @@ impl ExecutionPlan for CoalesceBatchesExec { } fn statistics_by_partition(&self) -> Result { - self.input - .statistics_by_partition()? - .into_iter() - .map(|stat| Statistics::with_fetch(stat, self.schema(), self.fetch, 0, 1)) - .collect() + let input_stats = self.input.statistics_by_partition()?; + + let stats: Result>> = input_stats + .iter() + .map(|stat| { + let fetched_stat = Statistics::with_fetch( + stat.clone(), + self.schema(), + self.fetch, + 0, + 1, + )?; + Ok(Arc::new(fetched_stat)) + }) + .collect(); + + Ok(PartitionedStatistics::new(stats?)) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 7835b97244737..8906aa1b04b69 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -31,6 +31,7 @@ use crate::execution_plan::CardinalityEffect; use crate::projection::{make_with_child, ProjectionExec}; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; +use crate::statistics::PartitionedStatistics; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; @@ -200,7 +201,9 @@ impl ExecutionPlan for CoalescePartitionsExec { } fn statistics_by_partition(&self) -> Result { - Ok(vec![self.statistics()?]) + Ok(PartitionedStatistics::new(vec![Arc::new( + self.statistics()?, + )])) } fn supports_limit_pushdown(&self) -> bool { diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 49aa46b59b577..d5878e12cf594 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -54,8 +54,8 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use datafusion_physical_expr_common::sort_expr::LexRequirement; -use futures::stream::{StreamExt, TryStreamExt}; use crate::statistics::PartitionedStatistics; +use futures::stream::{StreamExt, TryStreamExt}; /// Represent nodes in the DataFusion Physical Plan. /// @@ -436,8 +436,12 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// [`Statistics::new_unknown`] for each partition. fn statistics_by_partition(&self) -> Result { Ok(PartitionedStatistics::new(vec![ - Arc::new(Statistics::new_unknown(&self.schema())); - self.properties().partitioning.partition_count() + Arc::new( + Statistics::new_unknown(&self.schema()) + ); + self.properties() + .partitioning + .partition_count() ])) } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index a0c49347f0fb3..59df4a5f3f861 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -58,6 +58,7 @@ use datafusion_physical_expr::{ ExprBoundaries, PhysicalExpr, }; +use crate::statistics::PartitionedStatistics; use datafusion_physical_expr_common::physical_expr::fmt_sql; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -414,19 +415,23 @@ impl ExecutionPlan for FilterExec { } fn statistics_by_partition(&self) -> Result { - self.input - .statistics_by_partition()? - .into_iter() - .map(|input_stat| { - Self::statistics_helper( + let input_stats = self.input.statistics_by_partition()?; + + let stats: Result>> = input_stats + .iter() + .map(|stat| { + let stat = Self::statistics_helper( self.schema(), - input_stat, + stat.clone(), self.predicate(), self.default_selectivity, ) - .map(|stat| stat.project(self.projection.as_ref())) + .map(|stat| stat.project(self.projection.as_ref()))?; + Ok(Arc::new(stat)) }) - .collect() + .collect(); + + Ok(PartitionedStatistics::new(stats?)) } fn cardinality_effect(&self) -> CardinalityEffect { diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index a7c2f20c253b5..56c3f301fc20f 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -46,7 +46,7 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; -use crate::statistics::compute_summary_statistics; +use crate::statistics::{compute_summary_statistics, PartitionedStatistics}; use async_trait::async_trait; use futures::{ready, Stream, StreamExt, TryStreamExt}; @@ -349,19 +349,24 @@ impl ExecutionPlan for CrossJoinExec { let right_stats = self.right.statistics_by_partition()?; if left_stats.is_empty() || right_stats.is_empty() { - return Ok(vec![]); + return Ok(PartitionedStatistics::new(vec![])); } // Summarize the `left_stats` - let statistics = - compute_summary_statistics(left_stats, self.schema.fields().len(), |stats| { - Some(stats) - }); - - Ok(right_stats - .into_iter() - .map(|right| stats_cartesian_product(statistics.clone(), right)) - .collect()) + let statistics = compute_summary_statistics( + left_stats.iter(), + self.schema.fields().len(), + |stats| Some(stats), + ); + + Ok(PartitionedStatistics::new( + right_stats + .iter() + .map(|right| { + Arc::new(stats_cartesian_product(statistics.clone(), right.clone())) + }) + .collect(), + )) } /// Tries to swap the projection with its input [`CrossJoinExec`]. If it can be done, diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 00b4a68499839..b2eedff1a63b1 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -35,6 +35,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; +use crate::statistics::PartitionedStatistics; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -203,7 +204,9 @@ impl ExecutionPlan for GlobalLimitExec { } fn statistics_by_partition(&self) -> Result { - Ok(vec![self.statistics()?]) + Ok(PartitionedStatistics::new(vec![Arc::new( + self.statistics()?, + )])) } fn fetch(&self) -> Option { @@ -348,13 +351,23 @@ impl ExecutionPlan for LocalLimitExec { } fn statistics_by_partition(&self) -> Result { - self.input - .statistics_by_partition()? - .into_iter() - .map(|input_stat| { - Statistics::with_fetch(input_stat, self.schema(), Some(self.fetch), 0, 1) + let input_stats = self.input.statistics_by_partition()?; + + let stats: Result>> = input_stats + .iter() + .map(|stat| { + let fetched_stat = Statistics::with_fetch( + stat.clone(), + self.schema(), + Some(self.fetch), + 0, + 1, + )?; + Ok(Arc::new(fetched_stat)) }) - .collect() + .collect(); + + Ok(PartitionedStatistics::new(stats?)) } fn fetch(&self) -> Option { diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 789e80dea672f..831fb46ed3e86 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -48,6 +48,7 @@ use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::PhysicalExprRef; +use crate::statistics::PartitionedStatistics; use datafusion_physical_expr_common::physical_expr::fmt_sql; use futures::stream::{Stream, StreamExt}; use itertools::Itertools; @@ -252,18 +253,19 @@ impl ExecutionPlan for ProjectionExec { } fn statistics_by_partition(&self) -> Result { - Ok(self - .input - .statistics_by_partition()? - .into_iter() - .map(|input_stats| { - stats_projection( - input_stats, - self.expr.iter().map(|(e, _)| Arc::clone(e)), - Arc::clone(&self.schema), - ) - }) - .collect()) + Ok(PartitionedStatistics::new( + self.input + .statistics_by_partition()? + .iter() + .map(|input_stats| { + Arc::new(stats_projection( + input_stats.clone(), + self.expr.iter().map(|(e, _)| Arc::clone(e)), + Arc::clone(&self.schema), + )) + }) + .collect(), + )) } fn supports_limit_pushdown(&self) -> bool { diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 4d35f98b081c6..62b6e040031a9 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -60,6 +60,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::LexOrdering; use datafusion_physical_expr_common::sort_expr::LexRequirement; +use crate::statistics::PartitionedStatistics; use futures::{StreamExt, TryStreamExt}; use log::{debug, trace}; @@ -1302,13 +1303,27 @@ impl ExecutionPlan for SortExec { fn statistics_by_partition(&self) -> Result { if !self.preserve_partitioning() { - return Ok(vec![self.statistics()?]); + return Ok(PartitionedStatistics::new(vec![Arc::new( + self.statistics()?, + )])); } - self.input - .statistics_by_partition()? - .into_iter() - .map(|stat| Statistics::with_fetch(stat, self.schema(), self.fetch, 0, 1)) - .collect() + let input_stats = self.input.statistics_by_partition()?; + + let stats: Result>> = input_stats + .iter() + .map(|stat| { + let fetched_stat = Statistics::with_fetch( + stat.clone(), + self.schema(), + self.fetch, + 0, + 1, + )?; + Ok(Arc::new(fetched_stat)) + }) + .collect(); + + Ok(PartitionedStatistics::new(stats?)) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index b4825a1f59247..8d44962cdb33c 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -36,6 +36,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use crate::statistics::PartitionedStatistics; use log::{debug, trace}; /// Sort preserving merge execution plan @@ -347,7 +348,9 @@ impl ExecutionPlan for SortPreservingMergeExec { } fn statistics_by_partition(&self) -> Result { - Ok(vec![self.statistics()?]) + Ok(PartitionedStatistics::new(vec![Arc::new( + self.statistics()?, + )])) } fn supports_limit_pushdown(&self) -> bool { diff --git a/datafusion/physical-plan/src/statistics.rs b/datafusion/physical-plan/src/statistics.rs index 0d8f15f0947df..855e007e8709d 100644 --- a/datafusion/physical-plan/src/statistics.rs +++ b/datafusion/physical-plan/src/statistics.rs @@ -48,6 +48,10 @@ impl PartitionedStatistics { pub fn iter(&self) -> impl Iterator { self.inner.iter().map(|arc| arc.as_ref()) } + + pub fn is_empty(&self) -> bool { + self.inner.is_empty() + } } /// Generic function to compute statistics across multiple items that have statistics diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 45533c4cdfd81..c310ff5692e8d 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -46,6 +46,7 @@ use datafusion_common::{exec_err, internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{calculate_union, EquivalenceProperties}; +use crate::statistics::PartitionedStatistics; use futures::Stream; use itertools::Itertools; use log::{debug, trace, warn}; @@ -271,14 +272,18 @@ impl ExecutionPlan for UnionExec { } fn statistics_by_partition(&self) -> Result { - Ok(self + let input_stats_vec = self .inputs .iter() - .map(|child| child.statistics_by_partition()) - .collect::>>()? - .into_iter() - .flatten() - .collect()) + .map(|input| input.statistics_by_partition()) + .collect::>>()?; + + let all_stats: Vec<_> = input_stats_vec + .iter() + .flat_map(|input_stats| input_stats.iter().map(|stat| Arc::new(stat.clone()))) + .collect(); + + Ok(PartitionedStatistics::new(all_stats)) } fn benefits_from_input_partitioning(&self) -> Vec { 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 3653ee949c744..36dc2a3799c8d 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -63,6 +63,7 @@ use datafusion_physical_expr::window::{ use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use crate::statistics::PartitionedStatistics; use futures::stream::Stream; use futures::{ready, StreamExt}; use hashbrown::hash_table::HashTable; @@ -365,11 +366,15 @@ impl ExecutionPlan for BoundedWindowAggExec { } fn statistics_by_partition(&self) -> Result { - self.input - .statistics_by_partition()? - .into_iter() - .map(|stat| self.statistics_helper(stat)) - .collect() + let input_stats = self.input.statistics_by_partition()?; + let stats: Result>> = input_stats + .iter() + .map(|stat| { + let stat = self.statistics_helper(stat.clone())?; + Ok(Arc::new(stat)) + }) + .collect(); + Ok(PartitionedStatistics::new(stats?)) } } From 20f9af4da47aeac40097ede131b72862ce58e52b Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 7 Apr 2025 19:35:36 +0800 Subject: [PATCH 16/26] impl index and len for PartitionedStatistics --- datafusion/physical-plan/src/statistics.rs | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/datafusion/physical-plan/src/statistics.rs b/datafusion/physical-plan/src/statistics.rs index 855e007e8709d..d005d861c9321 100644 --- a/datafusion/physical-plan/src/statistics.rs +++ b/datafusion/physical-plan/src/statistics.rs @@ -21,6 +21,7 @@ use datafusion_common::stats::Precision; use datafusion_common::{ColumnStatistics, ScalarValue, Statistics}; use std::mem; +use std::ops::Index; use std::sync::Arc; /// Represents statistics data grouped by partition. @@ -52,6 +53,18 @@ impl PartitionedStatistics { pub fn is_empty(&self) -> bool { self.inner.is_empty() } + + pub fn len(&self) -> usize { + self.inner.len() + } +} + +impl Index for PartitionedStatistics { + type Output = Statistics; + + fn index(&self, partition_idx: usize) -> &Self::Output { + self.statistics(partition_idx) + } } /// Generic function to compute statistics across multiple items that have statistics From f9296f2ce456fad7ea37f9bf6e94d13f4f6d4008 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 7 Apr 2025 20:12:44 +0800 Subject: [PATCH 17/26] add test for cross join --- .../partition_statistics.rs | 67 +++++++++++++++++-- .../physical-plan/src/joins/cross_join.rs | 2 +- 2 files changed, 61 insertions(+), 8 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index 4292b9b3bb753..672dcf246b40e 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -29,6 +29,7 @@ mod test { use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_plan::filter::FilterExec; + use datafusion_physical_plan::joins::CrossJoinExec; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::union::UnionExec; @@ -48,6 +49,7 @@ mod test { /// @param target_partition Optional parameter to set the target partitions /// @return ExecutionPlan representing the scan of the table with statistics async fn create_scan_exec_with_statistics( + create_table_sql: Option<&str>, target_partition: Option, ) -> Arc { let mut session_config = SessionConfig::new().with_collect_statistics(true); @@ -56,14 +58,25 @@ mod test { } let ctx = SessionContext::new_with_config(session_config); // Create table with partition - let create_table_sql = "CREATE EXTERNAL TABLE t1 (id INT not null, date DATE) STORED AS PARQUET LOCATION './tests/data/test_statistics_per_partition' PARTITIONED BY (date) WITH ORDER (id ASC);"; + let create_table_sql = create_table_sql.unwrap_or( + "CREATE EXTERNAL TABLE t1 (id INT NOT NULL, date DATE) \ + STORED AS PARQUET LOCATION './tests/data/test_statistics_per_partition'\ + PARTITIONED BY (date) \ + WITH ORDER (id ASC);", + ); + // Get table name from `create_table_sql` + let table_name = create_table_sql + .split_whitespace() + .nth(3) + .unwrap_or("t1") + .to_string(); ctx.sql(create_table_sql) .await .unwrap() .collect() .await .unwrap(); - let table = ctx.table_provider("t1").await.unwrap(); + let table = ctx.table_provider(table_name.as_str()).await.unwrap(); let listing_table = table .as_any() .downcast_ref::() @@ -111,7 +124,7 @@ mod test { #[tokio::test] async fn test_statistics_by_partition_of_data_source() -> datafusion_common::Result<()> { - let scan = create_scan_exec_with_statistics(Some(2)).await; + let scan = create_scan_exec_with_statistics(None, Some(2)).await; let statistics = scan.statistics_by_partition()?; let expected_statistic_partition_1 = create_partition_statistics(2, 110, 3, 4, true); @@ -127,7 +140,7 @@ mod test { #[tokio::test] async fn test_statistics_by_partition_of_projection() -> datafusion_common::Result<()> { - let scan = create_scan_exec_with_statistics(Some(2)).await; + let scan = create_scan_exec_with_statistics(None, Some(2)).await; // Add projection execution plan let exprs: Vec<(Arc, String)> = vec![(Arc::new(Column::new("id", 0)), "id".to_string())]; @@ -146,7 +159,7 @@ mod test { #[tokio::test] async fn test_statistics_by_partition_of_sort() -> datafusion_common::Result<()> { - let scan = create_scan_exec_with_statistics(Some(2)).await; + let scan = create_scan_exec_with_statistics(None, Some(2)).await; // Add sort execution plan let sort = SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { @@ -179,7 +192,7 @@ mod test { #[tokio::test] async fn test_statistics_by_partition_of_filter() -> datafusion_common::Result<()> { - let scan = create_scan_exec_with_statistics(Some(2)).await; + let scan = create_scan_exec_with_statistics(None, Some(2)).await; let schema = Schema::new(vec![Field::new("id", DataType::Int32, false)]); let predicate = binary( Arc::new(Column::new("id", 0)), @@ -221,7 +234,7 @@ mod test { #[tokio::test] async fn test_statistic_by_partition_of_union() -> datafusion_common::Result<()> { - let scan = create_scan_exec_with_statistics(Some(2)).await; + let scan = create_scan_exec_with_statistics(None, Some(2)).await; let union_exec = Arc::new(UnionExec::new(vec![scan.clone(), scan])); let statistics = union_exec.statistics_by_partition()?; // Check that we have 4 partitions (2 from each scan) @@ -241,4 +254,44 @@ mod test { Ok(()) } + + #[tokio::test] + async fn test_statistic_by_partition_of_cross_join() -> datafusion_common::Result<()> + { + let left_scan = create_scan_exec_with_statistics(None, Some(2)).await; + let right_create_table_sql = "CREATE EXTERNAL TABLE t2 (id INT NOT NULL) \ + STORED AS PARQUET LOCATION './tests/data/test_statistics_per_partition'\ + WITH ORDER (id ASC);"; + let right_scan = + create_scan_exec_with_statistics(Some(right_create_table_sql), Some(2)).await; + let cross_join = CrossJoinExec::new(left_scan, right_scan); + let statistics = cross_join.statistics_by_partition()?; + // Check that we have 2 partitions + assert_eq!(statistics.len(), 2); + let mut expected_statistic_partition_1 = + create_partition_statistics(8, 48400, 1, 4, true); + expected_statistic_partition_1 + .column_statistics + .push(ColumnStatistics { + null_count: Precision::Exact(0), + max_value: Precision::Exact(ScalarValue::Int32(Some(4))), + min_value: Precision::Exact(ScalarValue::Int32(Some(3))), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + }); + let mut expected_statistic_partition_2 = + create_partition_statistics(8, 48400, 1, 4, true); + expected_statistic_partition_2 + .column_statistics + .push(ColumnStatistics { + null_count: Precision::Exact(0), + max_value: Precision::Exact(ScalarValue::Int32(Some(2))), + min_value: Precision::Exact(ScalarValue::Int32(Some(1))), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + }); + assert_eq!(statistics[0], expected_statistic_partition_1); + assert_eq!(statistics[1], expected_statistic_partition_2); + Ok(()) + } } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 56c3f301fc20f..4e14e41178b58 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -355,7 +355,7 @@ impl ExecutionPlan for CrossJoinExec { // Summarize the `left_stats` let statistics = compute_summary_statistics( left_stats.iter(), - self.schema.fields().len(), + self.left.schema().fields().len(), |stats| Some(stats), ); From 0503b1362b1ea08dcae6cf2210944dd389ff5bb3 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 7 Apr 2025 20:27:12 +0800 Subject: [PATCH 18/26] fix clippy --- datafusion/datasource/src/source.rs | 13 ++++++++----- datafusion/physical-plan/src/execution_plan.rs | 15 +++++++-------- 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 3958575949f5b..93a6f04451d7d 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -204,16 +204,19 @@ impl ExecutionPlan for DataSourceExec { fn statistics_by_partition( &self, ) -> datafusion_common::Result { - let mut statistics = vec![ - Arc::new(Statistics::new_unknown(&self.schema())); - self.properties().partitioning.partition_count() - ]; + let mut statistics = { + let mut v = + Vec::with_capacity(self.properties().partitioning.partition_count()); + (0..self.properties().partitioning.partition_count()) + .for_each(|_| v.push(Arc::new(Statistics::new_unknown(&self.schema())))); + v + }; if let Some(file_config) = self.data_source.as_any().downcast_ref::() { for (idx, file_group) in file_config.file_groups.iter().enumerate() { if let Some(stat) = file_group.statistics() { - statistics[idx] = stat.clone(); + statistics[idx] = Arc::clone(stat); } } } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index d5878e12cf594..cc17ebef26e89 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -435,14 +435,13 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// If statistics are not available, returns an array of /// [`Statistics::new_unknown`] for each partition. fn statistics_by_partition(&self) -> Result { - Ok(PartitionedStatistics::new(vec![ - Arc::new( - Statistics::new_unknown(&self.schema()) - ); - self.properties() - .partitioning - .partition_count() - ])) + Ok(PartitionedStatistics::new({ + let mut v = + Vec::with_capacity(self.properties().partitioning.partition_count()); + (0..self.properties().partitioning.partition_count()) + .for_each(|_| v.push(Arc::new(Statistics::new_unknown(&self.schema())))); + v + })) } /// Returns `true` if a limit can be safely pushed down through this From 513d9d14d8d4ec6039bfb3faf40da264ca746df1 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Tue, 8 Apr 2025 15:44:59 +0800 Subject: [PATCH 19/26] Check the statistics_by_partition with real results --- .../partition_statistics.rs | 191 ++++++++++++++++-- 1 file changed, 177 insertions(+), 14 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index 672dcf246b40e..38b8e86c4638c 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -17,23 +17,30 @@ #[cfg(test)] mod test { + use arrow::array::{Int32Array, RecordBatch}; use arrow_schema::{DataType, Field, Schema, SortOptions}; use datafusion::datasource::listing::ListingTable; use datafusion::prelude::SessionContext; use datafusion_catalog::TableProvider; use datafusion_common::stats::Precision; + use datafusion_common::Result; use datafusion_common::{ColumnStatistics, ScalarValue, Statistics}; use datafusion_execution::config::SessionConfig; + use datafusion_execution::TaskContext; use datafusion_expr_common::operator::Operator; use datafusion_physical_expr::expressions::{binary, lit, Column}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; + use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; + use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::CrossJoinExec; + use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::union::UnionExec; - use datafusion_physical_plan::ExecutionPlan; + use datafusion_physical_plan::{execute_stream_partitioned, ExecutionPlan}; + use futures::TryStreamExt; use std::sync::Arc; /// Creates a test table with statistics from the test data directory. @@ -121,9 +128,63 @@ mod test { } } + /// Helper function to validate that statistics from statistics_by_partition match the actual data + async fn validate_statistics_with_data( + plan: Arc, + expected_stats: Vec<(i32, i32, usize)>, // (min_id, max_id, row_count) + id_column_index: usize, + ) -> Result<()> { + let ctx = TaskContext::default(); + let partitions = execute_stream_partitioned(plan, Arc::new(ctx))?; + + let mut actual_stats = Vec::new(); + for partition_stream in partitions.into_iter() { + let result: Vec = partition_stream.try_collect().await?; + + let mut min_id = i32::MAX; + let mut max_id = i32::MIN; + let mut row_count = 0; + + for batch in result { + if batch.num_columns() > id_column_index { + let id_array = batch + .column(id_column_index) + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..batch.num_rows() { + let id_value = id_array.value(i); + min_id = min_id.min(id_value); + max_id = max_id.max(id_value); + row_count += 1; + } + } + } + + if row_count > 0 { + actual_stats.push((min_id, max_id, row_count)); + } + } + + // Compare actual data with expected statistics + assert_eq!( + actual_stats.len(), + expected_stats.len(), + "Number of partitions with data doesn't match expected" + ); + for i in 0..actual_stats.len() { + assert_eq!( + actual_stats[i], expected_stats[i], + "Partition {} data doesn't match statistics", + i + ); + } + + Ok(()) + } + #[tokio::test] - async fn test_statistics_by_partition_of_data_source() -> datafusion_common::Result<()> - { + async fn test_statistics_by_partition_of_data_source() -> Result<()> { let scan = create_scan_exec_with_statistics(None, Some(2)).await; let statistics = scan.statistics_by_partition()?; let expected_statistic_partition_1 = @@ -134,12 +195,19 @@ mod test { assert_eq!(statistics.len(), 2); assert_eq!(statistics[0], expected_statistic_partition_1); assert_eq!(statistics[1], expected_statistic_partition_2); + + // Check the statistics_by_partition with real results + let expected_stats = vec![ + (3, 4, 2), // (min_id, max_id, row_count) for first partition + (1, 2, 2), // (min_id, max_id, row_count) for second partition + ]; + validate_statistics_with_data(scan, expected_stats, 0).await?; + Ok(()) } #[tokio::test] - async fn test_statistics_by_partition_of_projection() -> datafusion_common::Result<()> - { + async fn test_statistics_by_partition_of_projection() -> Result<()> { let scan = create_scan_exec_with_statistics(None, Some(2)).await; // Add projection execution plan let exprs: Vec<(Arc, String)> = @@ -154,12 +222,16 @@ mod test { assert_eq!(statistics.len(), 2); assert_eq!(statistics[0], expected_statistic_partition_1); assert_eq!(statistics[1], expected_statistic_partition_2); + + // Check the statistics_by_partition with real results + let expected_stats = vec![(3, 4, 2), (1, 2, 2)]; + validate_statistics_with_data(Arc::new(projection), expected_stats, 0).await?; Ok(()) } #[tokio::test] - async fn test_statistics_by_partition_of_sort() -> datafusion_common::Result<()> { - let scan = create_scan_exec_with_statistics(None, Some(2)).await; + async fn test_statistics_by_partition_of_sort() -> Result<()> { + let scan_1 = create_scan_exec_with_statistics(None, Some(1)).await; // Add sort execution plan let sort = SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { @@ -169,16 +241,34 @@ mod test { nulls_first: false, }, }]), - scan, + scan_1, ); - let mut sort_exec = Arc::new(sort.clone()); + let sort_exec = Arc::new(sort.clone()); let statistics = sort_exec.statistics_by_partition()?; let expected_statistic_partition = create_partition_statistics(4, 220, 1, 4, true); assert_eq!(statistics.len(), 1); assert_eq!(statistics[0], expected_statistic_partition); + // Check the statistics_by_partition with real results + let expected_stats = vec![(1, 4, 4)]; + validate_statistics_with_data(sort_exec.clone(), expected_stats, 0).await?; - sort_exec = Arc::new(sort.with_preserve_partitioning(true)); + // Sort with preserve_partitioning + let scan_2 = create_scan_exec_with_statistics(None, Some(2)).await; + // Add sort execution plan + let sort_exec = Arc::new( + SortExec::new( + LexOrdering::new(vec![PhysicalSortExpr { + expr: Arc::new(Column::new("id", 0)), + options: SortOptions { + descending: false, + nulls_first: false, + }, + }]), + scan_2, + ) + .with_preserve_partitioning(true), + ); let expected_statistic_partition_1 = create_partition_statistics(2, 110, 3, 4, true); let expected_statistic_partition_2 = @@ -187,11 +277,15 @@ mod test { assert_eq!(statistics.len(), 2); assert_eq!(statistics[0], expected_statistic_partition_1); assert_eq!(statistics[1], expected_statistic_partition_2); + + // Check the statistics_by_partition with real results + let expected_stats = vec![(3, 4, 2), (1, 2, 2)]; + validate_statistics_with_data(sort_exec, expected_stats, 0).await?; Ok(()) } #[tokio::test] - async fn test_statistics_by_partition_of_filter() -> datafusion_common::Result<()> { + async fn test_statistics_by_partition_of_filter() -> Result<()> { let scan = create_scan_exec_with_statistics(None, Some(2)).await; let schema = Schema::new(vec![Field::new("id", DataType::Int32, false)]); let predicate = binary( @@ -233,7 +327,7 @@ mod test { } #[tokio::test] - async fn test_statistic_by_partition_of_union() -> datafusion_common::Result<()> { + async fn test_statistic_by_partition_of_union() -> Result<()> { let scan = create_scan_exec_with_statistics(None, Some(2)).await; let union_exec = Arc::new(UnionExec::new(vec![scan.clone(), scan])); let statistics = union_exec.statistics_by_partition()?; @@ -252,12 +346,14 @@ mod test { // Verify fourth partition (from second scan - same as second partition) assert_eq!(statistics[3], expected_statistic_partition_2); + // Check the statistics_by_partition with real results + let expected_stats = vec![(3, 4, 2), (1, 2, 2), (3, 4, 2), (1, 2, 2)]; + validate_statistics_with_data(union_exec, expected_stats, 0).await?; Ok(()) } #[tokio::test] - async fn test_statistic_by_partition_of_cross_join() -> datafusion_common::Result<()> - { + async fn test_statistic_by_partition_of_cross_join() -> Result<()> { let left_scan = create_scan_exec_with_statistics(None, Some(2)).await; let right_create_table_sql = "CREATE EXTERNAL TABLE t2 (id INT NOT NULL) \ STORED AS PARQUET LOCATION './tests/data/test_statistics_per_partition'\ @@ -292,6 +388,73 @@ mod test { }); assert_eq!(statistics[0], expected_statistic_partition_1); assert_eq!(statistics[1], expected_statistic_partition_2); + + // Check the statistics_by_partition with real results + let expected_stats = vec![(1, 4, 8), (1, 4, 8)]; + validate_statistics_with_data(Arc::new(cross_join), expected_stats, 0).await?; + Ok(()) + } + + #[tokio::test] + async fn test_statistic_by_partition_of_coalesce_batches() -> Result<()> { + let scan = create_scan_exec_with_statistics(None, Some(2)).await; + let coalesce_batches = CoalesceBatchesExec::new(scan, 2); + let expected_statistic_partition_1 = + create_partition_statistics(2, 110, 3, 4, true); + let expected_statistic_partition_2 = + create_partition_statistics(2, 110, 1, 2, true); + let statistics = coalesce_batches.statistics_by_partition()?; + assert_eq!(statistics.len(), 2); + assert_eq!(statistics[0], expected_statistic_partition_1); + assert_eq!(statistics[1], expected_statistic_partition_2); + + // Check the statistics_by_partition with real results + let expected_stats = vec![(3, 4, 2), (1, 2, 2)]; + validate_statistics_with_data(Arc::new(coalesce_batches), expected_stats, 0) + .await?; + Ok(()) + } + + #[tokio::test] + async fn test_statistic_by_partition_of_coalesce_partitions() -> Result<()> { + let scan = create_scan_exec_with_statistics(None, Some(2)).await; + let coalesce_partitions = CoalescePartitionsExec::new(scan); + let expected_statistic_partition = + create_partition_statistics(4, 220, 1, 4, true); + let statistics = coalesce_partitions.statistics_by_partition()?; + assert_eq!(statistics.len(), 1); + assert_eq!(statistics[0], expected_statistic_partition); + + // Check the statistics_by_partition with real results + let expected_stats = vec![(1, 4, 4)]; + validate_statistics_with_data(Arc::new(coalesce_partitions), expected_stats, 0) + .await?; + Ok(()) + } + + #[tokio::test] + async fn test_statistic_by_partition_of_local_limit() -> Result<()> { + let scan = create_scan_exec_with_statistics(None, Some(2)).await; + let local_limit = LocalLimitExec::new(scan.clone(), 1); + let statistics = local_limit.statistics_by_partition()?; + assert_eq!(statistics.len(), 2); + let schema = scan.schema(); + let mut expected_statistic_partition = Statistics::new_unknown(&schema); + expected_statistic_partition.num_rows = Precision::Exact(1); + assert_eq!(statistics[0], expected_statistic_partition); + assert_eq!(statistics[1], expected_statistic_partition); + Ok(()) + } + + #[tokio::test] + async fn test_statistic_by_partition_of_global_limit_partitions() -> Result<()> { + let scan = create_scan_exec_with_statistics(None, Some(2)).await; + let global_limit = GlobalLimitExec::new(scan.clone(), 0, Some(2)); + let statistics = global_limit.statistics_by_partition()?; + assert_eq!(statistics.len(), 1); + let mut expected_statistic_partition = Statistics::new_unknown(&scan.schema()); + expected_statistic_partition.num_rows = Precision::Exact(2); + assert_eq!(statistics[0], expected_statistic_partition); Ok(()) } } From a119f60c5235864ca39b7946bfc7106c3247b2dc Mon Sep 17 00:00:00 2001 From: xudong963 Date: Tue, 8 Apr 2025 16:08:24 +0800 Subject: [PATCH 20/26] rebase main and fix cross join test --- .../core/tests/physical_optimizer/partition_statistics.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index 38b8e86c4638c..22f1404b94f16 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -354,7 +354,7 @@ mod test { #[tokio::test] async fn test_statistic_by_partition_of_cross_join() -> Result<()> { - let left_scan = create_scan_exec_with_statistics(None, Some(2)).await; + let left_scan = create_scan_exec_with_statistics(None, Some(1)).await; let right_create_table_sql = "CREATE EXTERNAL TABLE t2 (id INT NOT NULL) \ STORED AS PARQUET LOCATION './tests/data/test_statistics_per_partition'\ WITH ORDER (id ASC);"; From 51c4465e8b2413faa36b9bafe76bdff5b0f42d1c Mon Sep 17 00:00:00 2001 From: xudong963 Date: Fri, 25 Apr 2025 13:58:08 +0800 Subject: [PATCH 21/26] resolve conflicts --- datafusion/core/src/datasource/listing/table.rs | 1 - datafusion/datasource/src/source.rs | 5 ++--- datafusion/datasource/src/statistics.rs | 5 +---- datafusion/physical-plan/src/coalesce_batches.rs | 1 + 4 files changed, 4 insertions(+), 8 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 25771fbcf40b6..b3e51e6ecd295 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -58,7 +58,6 @@ use datafusion_common::stats::Precision; use datafusion_datasource::compute_all_files_statistics; use datafusion_datasource::file_groups::FileGroup; use datafusion_physical_expr_common::sort_expr::LexRequirement; -use datafusion_physical_plan::statistics::add_row_stats; use futures::{future, stream, Stream, StreamExt, TryStreamExt}; use itertools::Itertools; use object_store::ObjectStore; diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 93a6f04451d7d..d5338ce4378fd 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -25,6 +25,7 @@ use std::sync::Arc; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::statistics::PartitionedStatistics; use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, }; @@ -201,9 +202,7 @@ impl ExecutionPlan for DataSourceExec { self.data_source.statistics() } - fn statistics_by_partition( - &self, - ) -> datafusion_common::Result { + fn statistics_by_partition(&self) -> Result { let mut statistics = { let mut v = Vec::with_capacity(self.properties().partitioning.partition_count()); diff --git a/datafusion/datasource/src/statistics.rs b/datafusion/datasource/src/statistics.rs index f26099a815995..192bb43c85953 100644 --- a/datafusion/datasource/src/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -36,9 +36,6 @@ use datafusion_common::stats::Precision; use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; use datafusion_physical_expr::{expressions::Column, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; -pub(crate) use datafusion_physical_plan::statistics::{ - add_row_stats, compute_summary_statistics, set_max_if_greater, set_min_if_lesser, -}; use datafusion_physical_plan::{ColumnStatistics, Statistics}; /// A normalized representation of file min/max statistics that allows for efficient sorting & comparison. @@ -479,7 +476,7 @@ pub fn compute_all_files_statistics( // Then summary statistics across all file groups let file_groups_statistics = file_groups_with_stats .iter() - .filter_map(|file_group| file_group.statistics()); + .filter_map(|file_group| file_group.statistics().as_deref()); let mut statistics = Statistics::try_merge_iter(file_groups_statistics, &table_schema)?; diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 3cebd53921a21..888cea6d9cdb0 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -24,6 +24,7 @@ use std::task::{Context, Poll}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, ExecutionPlanProperties, PlanProperties, Statistics}; +use crate::statistics::PartitionedStatistics; use crate::{ DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, }; From 3674acc79f2f607ab9aa889c1e345d747aafb021 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Fri, 25 Apr 2025 15:44:02 +0800 Subject: [PATCH 22/26] Feat: introduce partition statistics API --- .../datasource/physical_plan/arrow_file.rs | 5 +- .../partition_statistics.rs | 86 +++++--- datafusion/datasource-avro/src/source.rs | 5 +- datafusion/datasource-csv/src/source.rs | 5 +- datafusion/datasource/src/file_groups.rs | 5 - datafusion/datasource/src/source.rs | 29 ++- datafusion/datasource/src/statistics.rs | 2 +- .../physical-plan/src/coalesce_batches.rs | 24 +-- .../physical-plan/src/coalesce_partitions.rs | 7 +- .../physical-plan/src/execution_plan.rs | 36 ++-- datafusion/physical-plan/src/filter.rs | 28 +-- .../physical-plan/src/joins/cross_join.rs | 28 +-- datafusion/physical-plan/src/lib.rs | 1 - datafusion/physical-plan/src/limit.rs | 29 +-- datafusion/physical-plan/src/projection.rs | 20 +- datafusion/physical-plan/src/sorts/sort.rs | 32 +-- .../src/sorts/sort_preserving_merge.rs | 7 +- datafusion/physical-plan/src/statistics.rs | 196 ------------------ datafusion/physical-plan/src/union.rs | 17 +- .../src/windows/bounded_window_agg_exec.rs | 14 +- 20 files changed, 157 insertions(+), 419 deletions(-) delete mode 100644 datafusion/physical-plan/src/statistics.rs diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 5a237c29372f2..1ec31d0b2e3f6 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -42,7 +42,6 @@ use datafusion_physical_plan::{ }; use datafusion_datasource::file_groups::FileGroup; -use datafusion_physical_plan::statistics::PartitionedStatistics; use futures::StreamExt; use itertools::Itertools; use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore}; @@ -192,8 +191,8 @@ impl ExecutionPlan for ArrowExec { self.inner.statistics() } - fn statistics_by_partition(&self) -> Result { - self.inner.statistics_by_partition() + fn partition_statistics(&self, partition: Option) -> Result { + self.inner.partition_statistics(partition) } fn fetch(&self) -> Option { diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index 22f1404b94f16..b1bd3cc7f19a0 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -39,7 +39,9 @@ mod test { use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::union::UnionExec; - use datafusion_physical_plan::{execute_stream_partitioned, ExecutionPlan}; + use datafusion_physical_plan::{ + execute_stream_partitioned, ExecutionPlan, ExecutionPlanProperties, + }; use futures::TryStreamExt; use std::sync::Arc; @@ -186,7 +188,9 @@ mod test { #[tokio::test] async fn test_statistics_by_partition_of_data_source() -> Result<()> { let scan = create_scan_exec_with_statistics(None, Some(2)).await; - let statistics = scan.statistics_by_partition()?; + let statistics = (0..scan.output_partitioning().partition_count()) + .map(|idx| scan.partition_statistics(Some(idx))) + .collect::>>()?; let expected_statistic_partition_1 = create_partition_statistics(2, 110, 3, 4, true); let expected_statistic_partition_2 = @@ -212,8 +216,11 @@ mod test { // Add projection execution plan let exprs: Vec<(Arc, String)> = vec![(Arc::new(Column::new("id", 0)), "id".to_string())]; - let projection = ProjectionExec::try_new(exprs, scan)?; - let statistics = projection.statistics_by_partition()?; + let projection: Arc = + Arc::new(ProjectionExec::try_new(exprs, scan)?); + let statistics = (0..projection.output_partitioning().partition_count()) + .map(|idx| projection.partition_statistics(Some(idx))) + .collect::>>()?; let expected_statistic_partition_1 = create_partition_statistics(2, 8, 3, 4, false); let expected_statistic_partition_2 = @@ -225,7 +232,7 @@ mod test { // Check the statistics_by_partition with real results let expected_stats = vec![(3, 4, 2), (1, 2, 2)]; - validate_statistics_with_data(Arc::new(projection), expected_stats, 0).await?; + validate_statistics_with_data(projection, expected_stats, 0).await?; Ok(()) } @@ -243,8 +250,10 @@ mod test { }]), scan_1, ); - let sort_exec = Arc::new(sort.clone()); - let statistics = sort_exec.statistics_by_partition()?; + let sort_exec: Arc = Arc::new(sort.clone()); + let statistics = (0..sort_exec.output_partitioning().partition_count()) + .map(|idx| sort_exec.partition_statistics(Some(idx))) + .collect::>>()?; let expected_statistic_partition = create_partition_statistics(4, 220, 1, 4, true); assert_eq!(statistics.len(), 1); @@ -256,7 +265,7 @@ mod test { // Sort with preserve_partitioning let scan_2 = create_scan_exec_with_statistics(None, Some(2)).await; // Add sort execution plan - let sort_exec = Arc::new( + let sort_exec: Arc = Arc::new( SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("id", 0)), @@ -273,7 +282,9 @@ mod test { create_partition_statistics(2, 110, 3, 4, true); let expected_statistic_partition_2 = create_partition_statistics(2, 110, 1, 2, true); - let statistics = sort_exec.statistics_by_partition()?; + let statistics = (0..sort_exec.output_partitioning().partition_count()) + .map(|idx| sort_exec.partition_statistics(Some(idx))) + .collect::>>()?; assert_eq!(statistics.len(), 2); assert_eq!(statistics[0], expected_statistic_partition_1); assert_eq!(statistics[1], expected_statistic_partition_2); @@ -296,7 +307,7 @@ mod test { )?; let filter: Arc = Arc::new(FilterExec::try_new(predicate, scan)?); - let full_statistics = filter.statistics()?; + let full_statistics = filter.partition_statistics(None)?; let expected_full_statistic = Statistics { num_rows: Precision::Inexact(0), total_byte_size: Precision::Inexact(0), @@ -319,7 +330,9 @@ mod test { }; assert_eq!(full_statistics, expected_full_statistic); - let statistics = filter.statistics_by_partition()?; + let statistics = (0..filter.output_partitioning().partition_count()) + .map(|idx| filter.partition_statistics(Some(idx))) + .collect::>>()?; assert_eq!(statistics.len(), 2); assert_eq!(statistics[0], expected_full_statistic); assert_eq!(statistics[1], expected_full_statistic); @@ -329,8 +342,11 @@ mod test { #[tokio::test] async fn test_statistic_by_partition_of_union() -> Result<()> { let scan = create_scan_exec_with_statistics(None, Some(2)).await; - let union_exec = Arc::new(UnionExec::new(vec![scan.clone(), scan])); - let statistics = union_exec.statistics_by_partition()?; + let union_exec: Arc = + Arc::new(UnionExec::new(vec![scan.clone(), scan])); + let statistics = (0..union_exec.output_partitioning().partition_count()) + .map(|idx| union_exec.partition_statistics(Some(idx))) + .collect::>>()?; // Check that we have 4 partitions (2 from each scan) assert_eq!(statistics.len(), 4); let expected_statistic_partition_1 = @@ -360,8 +376,11 @@ mod test { WITH ORDER (id ASC);"; let right_scan = create_scan_exec_with_statistics(Some(right_create_table_sql), Some(2)).await; - let cross_join = CrossJoinExec::new(left_scan, right_scan); - let statistics = cross_join.statistics_by_partition()?; + let cross_join: Arc = + Arc::new(CrossJoinExec::new(left_scan, right_scan)); + let statistics = (0..cross_join.output_partitioning().partition_count()) + .map(|idx| cross_join.partition_statistics(Some(idx))) + .collect::>>()?; // Check that we have 2 partitions assert_eq!(statistics.len(), 2); let mut expected_statistic_partition_1 = @@ -391,52 +410,60 @@ mod test { // Check the statistics_by_partition with real results let expected_stats = vec![(1, 4, 8), (1, 4, 8)]; - validate_statistics_with_data(Arc::new(cross_join), expected_stats, 0).await?; + validate_statistics_with_data(cross_join, expected_stats, 0).await?; Ok(()) } #[tokio::test] async fn test_statistic_by_partition_of_coalesce_batches() -> Result<()> { let scan = create_scan_exec_with_statistics(None, Some(2)).await; - let coalesce_batches = CoalesceBatchesExec::new(scan, 2); + dbg!(scan.partition_statistics(Some(0))?); + let coalesce_batches: Arc = + Arc::new(CoalesceBatchesExec::new(scan, 2)); let expected_statistic_partition_1 = create_partition_statistics(2, 110, 3, 4, true); let expected_statistic_partition_2 = create_partition_statistics(2, 110, 1, 2, true); - let statistics = coalesce_batches.statistics_by_partition()?; + let statistics = (0..coalesce_batches.output_partitioning().partition_count()) + .map(|idx| coalesce_batches.partition_statistics(Some(idx))) + .collect::>>()?; assert_eq!(statistics.len(), 2); assert_eq!(statistics[0], expected_statistic_partition_1); assert_eq!(statistics[1], expected_statistic_partition_2); // Check the statistics_by_partition with real results let expected_stats = vec![(3, 4, 2), (1, 2, 2)]; - validate_statistics_with_data(Arc::new(coalesce_batches), expected_stats, 0) - .await?; + validate_statistics_with_data(coalesce_batches, expected_stats, 0).await?; Ok(()) } #[tokio::test] async fn test_statistic_by_partition_of_coalesce_partitions() -> Result<()> { let scan = create_scan_exec_with_statistics(None, Some(2)).await; - let coalesce_partitions = CoalescePartitionsExec::new(scan); + let coalesce_partitions: Arc = + Arc::new(CoalescePartitionsExec::new(scan)); let expected_statistic_partition = create_partition_statistics(4, 220, 1, 4, true); - let statistics = coalesce_partitions.statistics_by_partition()?; + let statistics = (0..coalesce_partitions.output_partitioning().partition_count()) + .map(|idx| coalesce_partitions.partition_statistics(Some(idx))) + .collect::>>()?; assert_eq!(statistics.len(), 1); assert_eq!(statistics[0], expected_statistic_partition); // Check the statistics_by_partition with real results let expected_stats = vec![(1, 4, 4)]; - validate_statistics_with_data(Arc::new(coalesce_partitions), expected_stats, 0) - .await?; + validate_statistics_with_data(coalesce_partitions, expected_stats, 0).await?; Ok(()) } #[tokio::test] async fn test_statistic_by_partition_of_local_limit() -> Result<()> { let scan = create_scan_exec_with_statistics(None, Some(2)).await; - let local_limit = LocalLimitExec::new(scan.clone(), 1); - let statistics = local_limit.statistics_by_partition()?; + let local_limit: Arc = + Arc::new(LocalLimitExec::new(scan.clone(), 1)); + let statistics = (0..local_limit.output_partitioning().partition_count()) + .map(|idx| local_limit.partition_statistics(Some(idx))) + .collect::>>()?; assert_eq!(statistics.len(), 2); let schema = scan.schema(); let mut expected_statistic_partition = Statistics::new_unknown(&schema); @@ -449,8 +476,11 @@ mod test { #[tokio::test] async fn test_statistic_by_partition_of_global_limit_partitions() -> Result<()> { let scan = create_scan_exec_with_statistics(None, Some(2)).await; - let global_limit = GlobalLimitExec::new(scan.clone(), 0, Some(2)); - let statistics = global_limit.statistics_by_partition()?; + let global_limit: Arc = + Arc::new(GlobalLimitExec::new(scan.clone(), 0, Some(2))); + let statistics = (0..global_limit.output_partitioning().partition_count()) + .map(|idx| global_limit.partition_statistics(Some(idx))) + .collect::>>()?; assert_eq!(statistics.len(), 1); let mut expected_statistic_partition = Statistics::new_unknown(&scan.schema()); expected_statistic_partition.num_rows = Precision::Exact(2); diff --git a/datafusion/datasource-avro/src/source.rs b/datafusion/datasource-avro/src/source.rs index 1a0feea3f1535..2ec9fc6a98b02 100644 --- a/datafusion/datasource-avro/src/source.rs +++ b/datafusion/datasource-avro/src/source.rs @@ -40,7 +40,6 @@ use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, }; -use datafusion_physical_plan::statistics::PartitionedStatistics; use object_store::ObjectStore; /// Execution plan for scanning Avro data source @@ -142,8 +141,8 @@ impl ExecutionPlan for AvroExec { self.inner.statistics() } - fn statistics_by_partition(&self) -> Result { - self.inner.statistics_by_partition() + fn partition_statistics(&self, partition: Option) -> Result { + self.inner.partition_statistics(partition) } fn metrics(&self) -> Option { diff --git a/datafusion/datasource-csv/src/source.rs b/datafusion/datasource-csv/src/source.rs index 28aad5e43f147..2c7a851159edf 100644 --- a/datafusion/datasource-csv/src/source.rs +++ b/datafusion/datasource-csv/src/source.rs @@ -51,7 +51,6 @@ use datafusion_physical_plan::{ use crate::file_format::CsvDecoder; use datafusion_datasource::file_groups::FileGroup; -use datafusion_physical_plan::statistics::PartitionedStatistics; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; use object_store::{GetOptions, GetResultPayload, ObjectStore}; @@ -382,8 +381,8 @@ impl ExecutionPlan for CsvExec { self.inner.statistics() } - fn statistics_by_partition(&self) -> Result { - self.inner.statistics_by_partition() + fn partition_statistics(&self, partition: Option) -> Result { + self.inner.partition_statistics(partition) } fn metrics(&self) -> Option { diff --git a/datafusion/datasource/src/file_groups.rs b/datafusion/datasource/src/file_groups.rs index ebf53d915777f..cb628147d088e 100644 --- a/datafusion/datasource/src/file_groups.rs +++ b/datafusion/datasource/src/file_groups.rs @@ -420,11 +420,6 @@ impl FileGroup { self.files.push(file); } - /// Get the statistics for this group - pub fn statistics(&self) -> &Option> { - &self.statistics - } - /// Get the statistics for this group pub fn statistics_ref(&self) -> Option<&Statistics> { self.statistics.as_deref() diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index d5338ce4378fd..a9ada9ff4de47 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -25,7 +25,6 @@ use std::sync::Arc; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_plan::projection::ProjectionExec; -use datafusion_physical_plan::statistics::PartitionedStatistics; use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, }; @@ -202,24 +201,22 @@ impl ExecutionPlan for DataSourceExec { self.data_source.statistics() } - fn statistics_by_partition(&self) -> Result { - let mut statistics = { - let mut v = - Vec::with_capacity(self.properties().partitioning.partition_count()); - (0..self.properties().partitioning.partition_count()) - .for_each(|_| v.push(Arc::new(Statistics::new_unknown(&self.schema())))); - v - }; - if let Some(file_config) = - self.data_source.as_any().downcast_ref::() - { - for (idx, file_group) in file_config.file_groups.iter().enumerate() { - if let Some(stat) = file_group.statistics() { - statistics[idx] = Arc::clone(stat); + fn partition_statistics(&self, partition: Option) -> Result { + if let Some(partition) = partition { + let mut statistics = Statistics::new_unknown(&self.schema()); + if let Some(file_config) = + self.data_source.as_any().downcast_ref::() + { + if let Some(file_group) = file_config.file_groups.get(partition) { + if let Some(stat) = file_group.statistics_ref() { + statistics = stat.clone(); + } } } + Ok(statistics) + } else { + Ok(self.data_source.statistics()?) } - Ok(PartitionedStatistics::new(statistics)) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/datasource/src/statistics.rs b/datafusion/datasource/src/statistics.rs index 192bb43c85953..dd9cd87e41406 100644 --- a/datafusion/datasource/src/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -476,7 +476,7 @@ pub fn compute_all_files_statistics( // Then summary statistics across all file groups let file_groups_statistics = file_groups_with_stats .iter() - .filter_map(|file_group| file_group.statistics().as_deref()); + .filter_map(|file_group| file_group.statistics_ref()); let mut statistics = Statistics::try_merge_iter(file_groups_statistics, &table_schema)?; diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 888cea6d9cdb0..09f06fe0f04b7 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -24,7 +24,6 @@ use std::task::{Context, Poll}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, ExecutionPlanProperties, PlanProperties, Statistics}; -use crate::statistics::PartitionedStatistics; use crate::{ DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, }; @@ -200,24 +199,11 @@ impl ExecutionPlan for CoalesceBatchesExec { Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } - fn statistics_by_partition(&self) -> Result { - let input_stats = self.input.statistics_by_partition()?; - - let stats: Result>> = input_stats - .iter() - .map(|stat| { - let fetched_stat = Statistics::with_fetch( - stat.clone(), - self.schema(), - self.fetch, - 0, - 1, - )?; - Ok(Arc::new(fetched_stat)) - }) - .collect(); - - Ok(PartitionedStatistics::new(stats?)) + fn partition_statistics(&self, partition: Option) -> Result { + let input_stats = self.input.partition_statistics(partition)?; + let fetched_stat = + Statistics::with_fetch(input_stats.clone(), self.schema(), self.fetch, 0, 1)?; + Ok(fetched_stat) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 8906aa1b04b69..9f210c3a2abc6 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -31,7 +31,6 @@ use crate::execution_plan::CardinalityEffect; use crate::projection::{make_with_child, ProjectionExec}; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; -use crate::statistics::PartitionedStatistics; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; @@ -200,10 +199,8 @@ impl ExecutionPlan for CoalescePartitionsExec { Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } - fn statistics_by_partition(&self) -> Result { - Ok(PartitionedStatistics::new(vec![Arc::new( - self.statistics()?, - )])) + fn partition_statistics(&self, _partition: Option) -> Result { + Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } fn supports_limit_pushdown(&self) -> bool { diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index cc17ebef26e89..f26fb788cae64 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -54,7 +54,6 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use datafusion_physical_expr_common::sort_expr::LexRequirement; -use crate::statistics::PartitionedStatistics; use futures::stream::{StreamExt, TryStreamExt}; /// Represent nodes in the DataFusion Physical Plan. @@ -431,17 +430,30 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { Ok(Statistics::new_unknown(&self.schema())) } - /// Returns statistics for each partition of this `ExecutionPlan` node. - /// If statistics are not available, returns an array of - /// [`Statistics::new_unknown`] for each partition. - fn statistics_by_partition(&self) -> Result { - Ok(PartitionedStatistics::new({ - let mut v = - Vec::with_capacity(self.properties().partitioning.partition_count()); - (0..self.properties().partitioning.partition_count()) - .for_each(|_| v.push(Arc::new(Statistics::new_unknown(&self.schema())))); - v - })) + /// Returns statistics for a specific partition of this `ExecutionPlan` node. + /// If statistics are not available, should return [`Statistics::new_unknown`] + /// (the default), not an error. + /// If `partition` is `None`, it returns statistics for the entire plan. + fn partition_statistics(&self, partition: Option) -> Result { + match partition { + Some(idx) => { + // Validate partition index + let partition_count = self.properties().partitioning.partition_count(); + if idx >= partition_count { + return internal_err!( + "Invalid partition index: {}, the partition count is {}", + idx, + partition_count + ); + } + // Default implementation: return unknown statistics for the specific partition + Ok(Statistics::new_unknown(&self.schema())) + } + None => { + // Return unknown statistics for the entire plan + Ok(Statistics::new_unknown(&self.schema())) + } + } } /// Returns `true` if a limit can be safely pushed down through this diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 59df4a5f3f861..848bb0e802cf0 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -58,7 +58,6 @@ use datafusion_physical_expr::{ ExprBoundaries, PhysicalExpr, }; -use crate::statistics::PartitionedStatistics; use datafusion_physical_expr_common::physical_expr::fmt_sql; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -414,24 +413,15 @@ impl ExecutionPlan for FilterExec { Ok(stats.project(self.projection.as_ref())) } - fn statistics_by_partition(&self) -> Result { - let input_stats = self.input.statistics_by_partition()?; - - let stats: Result>> = input_stats - .iter() - .map(|stat| { - let stat = Self::statistics_helper( - self.schema(), - stat.clone(), - self.predicate(), - self.default_selectivity, - ) - .map(|stat| stat.project(self.projection.as_ref()))?; - Ok(Arc::new(stat)) - }) - .collect(); - - Ok(PartitionedStatistics::new(stats?)) + fn partition_statistics(&self, partition: Option) -> Result { + let input_stats = self.input.partition_statistics(partition)?; + let stats = Self::statistics_helper( + self.schema(), + input_stats, + self.predicate(), + self.default_selectivity, + )?; + Ok(stats.project(self.projection.as_ref())) } fn cardinality_effect(&self) -> CardinalityEffect { diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 4e14e41178b58..fb0de0f3a1a59 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -46,7 +46,6 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; -use crate::statistics::{compute_summary_statistics, PartitionedStatistics}; use async_trait::async_trait; use futures::{ready, Stream, StreamExt, TryStreamExt}; @@ -344,29 +343,12 @@ impl ExecutionPlan for CrossJoinExec { )) } - fn statistics_by_partition(&self) -> Result { - let left_stats = self.left.statistics_by_partition()?; - let right_stats = self.right.statistics_by_partition()?; + fn partition_statistics(&self, partition: Option) -> Result { + // Get the all partitions statistics of the left + let left_stats = self.left.partition_statistics(None)?; + let right_stats = self.right.partition_statistics(partition)?; - if left_stats.is_empty() || right_stats.is_empty() { - return Ok(PartitionedStatistics::new(vec![])); - } - - // Summarize the `left_stats` - let statistics = compute_summary_statistics( - left_stats.iter(), - self.left.schema().fields().len(), - |stats| Some(stats), - ); - - Ok(PartitionedStatistics::new( - right_stats - .iter() - .map(|right| { - Arc::new(stats_cartesian_product(statistics.clone(), right.clone())) - }) - .collect(), - )) + Ok(stats_cartesian_product(left_stats, right_stats)) } /// Tries to swap the projection with its input [`CrossJoinExec`]. If it can be done, diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 4493569ae6ffd..a1862554b303e 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -92,6 +92,5 @@ pub mod udaf { } pub mod coalesce; -pub mod statistics; #[cfg(test)] pub mod test; diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index b2eedff1a63b1..3d14eb41584bb 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -35,7 +35,6 @@ use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; -use crate::statistics::PartitionedStatistics; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -203,10 +202,9 @@ impl ExecutionPlan for GlobalLimitExec { ) } - fn statistics_by_partition(&self) -> Result { - Ok(PartitionedStatistics::new(vec![Arc::new( - self.statistics()?, - )])) + fn partition_statistics(&self, partition: Option) -> Result { + let input_stats = self.input.partition_statistics(partition)?; + Statistics::with_fetch(input_stats, self.schema(), self.fetch, self.skip, 1) } fn fetch(&self) -> Option { @@ -350,24 +348,9 @@ impl ExecutionPlan for LocalLimitExec { ) } - fn statistics_by_partition(&self) -> Result { - let input_stats = self.input.statistics_by_partition()?; - - let stats: Result>> = input_stats - .iter() - .map(|stat| { - let fetched_stat = Statistics::with_fetch( - stat.clone(), - self.schema(), - Some(self.fetch), - 0, - 1, - )?; - Ok(Arc::new(fetched_stat)) - }) - .collect(); - - Ok(PartitionedStatistics::new(stats?)) + fn partition_statistics(&self, partition: Option) -> Result { + let input_stats = self.input.partition_statistics(partition)?; + Statistics::with_fetch(input_stats, self.schema(), Some(self.fetch), 0, 1) } fn fetch(&self) -> Option { diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 831fb46ed3e86..6db545f519b78 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -48,7 +48,6 @@ use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::PhysicalExprRef; -use crate::statistics::PartitionedStatistics; use datafusion_physical_expr_common::physical_expr::fmt_sql; use futures::stream::{Stream, StreamExt}; use itertools::Itertools; @@ -252,19 +251,12 @@ impl ExecutionPlan for ProjectionExec { )) } - fn statistics_by_partition(&self) -> Result { - Ok(PartitionedStatistics::new( - self.input - .statistics_by_partition()? - .iter() - .map(|input_stats| { - Arc::new(stats_projection( - input_stats.clone(), - self.expr.iter().map(|(e, _)| Arc::clone(e)), - Arc::clone(&self.schema), - )) - }) - .collect(), + fn partition_statistics(&self, partition: Option) -> Result { + let input_stats = self.input.partition_statistics(partition)?; + Ok(stats_projection( + input_stats, + self.expr.iter().map(|(e, _)| Arc::clone(e)), + Arc::clone(&self.schema), )) } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 62b6e040031a9..facdd1eea94ed 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -60,7 +60,6 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::LexOrdering; use datafusion_physical_expr_common::sort_expr::LexRequirement; -use crate::statistics::PartitionedStatistics; use futures::{StreamExt, TryStreamExt}; use log::{debug, trace}; @@ -1301,29 +1300,18 @@ impl ExecutionPlan for SortExec { Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } - fn statistics_by_partition(&self) -> Result { + fn partition_statistics(&self, partition: Option) -> Result { if !self.preserve_partitioning() { - return Ok(PartitionedStatistics::new(vec![Arc::new( - self.statistics()?, - )])); + return Statistics::with_fetch( + self.input.partition_statistics(None)?, + self.schema(), + self.fetch, + 0, + 1, + ); } - let input_stats = self.input.statistics_by_partition()?; - - let stats: Result>> = input_stats - .iter() - .map(|stat| { - let fetched_stat = Statistics::with_fetch( - stat.clone(), - self.schema(), - self.fetch, - 0, - 1, - )?; - Ok(Arc::new(fetched_stat)) - }) - .collect(); - - Ok(PartitionedStatistics::new(stats?)) + let input_stats = self.input.partition_statistics(partition)?; + Statistics::with_fetch(input_stats, self.schema(), self.fetch, 0, 1) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 8d44962cdb33c..4ca522b267d45 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -36,7 +36,6 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; -use crate::statistics::PartitionedStatistics; use log::{debug, trace}; /// Sort preserving merge execution plan @@ -347,10 +346,8 @@ impl ExecutionPlan for SortPreservingMergeExec { self.input.statistics() } - fn statistics_by_partition(&self) -> Result { - Ok(PartitionedStatistics::new(vec![Arc::new( - self.statistics()?, - )])) + fn partition_statistics(&self, _partition: Option) -> Result { + self.input.partition_statistics(None) } fn supports_limit_pushdown(&self) -> bool { diff --git a/datafusion/physical-plan/src/statistics.rs b/datafusion/physical-plan/src/statistics.rs deleted file mode 100644 index d005d861c9321..0000000000000 --- a/datafusion/physical-plan/src/statistics.rs +++ /dev/null @@ -1,196 +0,0 @@ -// 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. - -//! Defines the cross join plan for loading the left side of the cross join -//! and producing batches in parallel for the right partitions - -use datafusion_common::stats::Precision; -use datafusion_common::{ColumnStatistics, ScalarValue, Statistics}; -use std::mem; -use std::ops::Index; -use std::sync::Arc; - -/// Represents statistics data grouped by partition. -/// -/// This structure maintains a collection of statistics, one for each partition -/// of a distributed dataset, allowing access to statistics by partition index. -#[derive(Debug, Clone)] -pub struct PartitionedStatistics { - inner: Vec>, -} - -impl PartitionedStatistics { - pub fn new(statistics: Vec>) -> Self { - Self { inner: statistics } - } - - pub fn statistics(&self, partition_idx: usize) -> &Statistics { - &self.inner[partition_idx] - } - - pub fn get_statistics(&self, partition_idx: usize) -> Option<&Statistics> { - self.inner.get(partition_idx).map(|arc| arc.as_ref()) - } - - pub fn iter(&self) -> impl Iterator { - self.inner.iter().map(|arc| arc.as_ref()) - } - - pub fn is_empty(&self) -> bool { - self.inner.is_empty() - } - - pub fn len(&self) -> usize { - self.inner.len() - } -} - -impl Index for PartitionedStatistics { - type Output = Statistics; - - fn index(&self, partition_idx: usize) -> &Self::Output { - self.statistics(partition_idx) - } -} - -/// Generic function to compute statistics across multiple items that have statistics -pub fn compute_summary_statistics( - items: I, - column_count: usize, - stats_extractor: impl Fn(&T) -> Option<&Statistics>, -) -> Statistics -where - I: IntoIterator, -{ - let mut col_stats_set = vec![ColumnStatistics::default(); column_count]; - let mut num_rows = Precision::::Absent; - let mut total_byte_size = Precision::::Absent; - - for (idx, item) in items.into_iter().enumerate() { - if let Some(item_stats) = stats_extractor(&item) { - if idx == 0 { - // First item, set values directly - num_rows = item_stats.num_rows; - total_byte_size = item_stats.total_byte_size; - for (index, column_stats) in - item_stats.column_statistics.iter().enumerate() - { - col_stats_set[index].null_count = column_stats.null_count; - col_stats_set[index].max_value = column_stats.max_value.clone(); - col_stats_set[index].min_value = column_stats.min_value.clone(); - col_stats_set[index].sum_value = column_stats.sum_value.clone(); - } - continue; - } - - // Accumulate statistics for subsequent items - num_rows = add_row_stats(item_stats.num_rows, num_rows); - total_byte_size = add_row_stats(item_stats.total_byte_size, total_byte_size); - - for (item_col_stats, col_stats) in item_stats - .column_statistics - .iter() - .zip(col_stats_set.iter_mut()) - { - col_stats.null_count = - add_row_stats(item_col_stats.null_count, col_stats.null_count); - set_max_if_greater(&item_col_stats.max_value, &mut col_stats.max_value); - set_min_if_lesser(&item_col_stats.min_value, &mut col_stats.min_value); - col_stats.sum_value = item_col_stats.sum_value.add(&col_stats.sum_value); - } - } - } - - Statistics { - num_rows, - total_byte_size, - column_statistics: col_stats_set, - } -} - -/// If the given value is numerically greater than the original maximum value, -/// return the new maximum value with appropriate exactness information. -pub fn set_max_if_greater( - max_nominee: &Precision, - max_value: &mut Precision, -) { - match (&max_value, max_nominee) { - (Precision::Exact(val1), Precision::Exact(val2)) if val1 < val2 => { - *max_value = max_nominee.clone(); - } - (Precision::Exact(val1), Precision::Inexact(val2)) - | (Precision::Inexact(val1), Precision::Inexact(val2)) - | (Precision::Inexact(val1), Precision::Exact(val2)) - if val1 < val2 => - { - *max_value = max_nominee.clone().to_inexact(); - } - (Precision::Exact(_), Precision::Absent) => { - let exact_max = mem::take(max_value); - *max_value = exact_max.to_inexact(); - } - (Precision::Absent, Precision::Exact(_)) => { - *max_value = max_nominee.clone().to_inexact(); - } - (Precision::Absent, Precision::Inexact(_)) => { - *max_value = max_nominee.clone(); - } - _ => {} - } -} - -/// If the given value is numerically lesser than the original minimum value, -/// return the new minimum value with appropriate exactness information. -pub fn set_min_if_lesser( - min_nominee: &Precision, - min_value: &mut Precision, -) { - match (&min_value, min_nominee) { - (Precision::Exact(val1), Precision::Exact(val2)) if val1 > val2 => { - *min_value = min_nominee.clone(); - } - (Precision::Exact(val1), Precision::Inexact(val2)) - | (Precision::Inexact(val1), Precision::Inexact(val2)) - | (Precision::Inexact(val1), Precision::Exact(val2)) - if val1 > val2 => - { - *min_value = min_nominee.clone().to_inexact(); - } - (Precision::Exact(_), Precision::Absent) => { - let exact_min = mem::take(min_value); - *min_value = exact_min.to_inexact(); - } - (Precision::Absent, Precision::Exact(_)) => { - *min_value = min_nominee.clone().to_inexact(); - } - (Precision::Absent, Precision::Inexact(_)) => { - *min_value = min_nominee.clone(); - } - _ => {} - } -} - -pub fn add_row_stats( - file_num_rows: Precision, - num_rows: Precision, -) -> Precision { - match (file_num_rows, &num_rows) { - (Precision::Absent, _) => num_rows.to_inexact(), - (lhs, Precision::Absent) => lhs.to_inexact(), - (lhs, rhs) => lhs.add(rhs), - } -} diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index c310ff5692e8d..5b9e18bdbd7b8 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -46,7 +46,6 @@ use datafusion_common::{exec_err, internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{calculate_union, EquivalenceProperties}; -use crate::statistics::PartitionedStatistics; use futures::Stream; use itertools::Itertools; use log::{debug, trace, warn}; @@ -271,19 +270,17 @@ impl ExecutionPlan for UnionExec { .unwrap_or_else(|| Statistics::new_unknown(&self.schema()))) } - fn statistics_by_partition(&self) -> Result { - let input_stats_vec = self + fn partition_statistics(&self, partition: Option) -> Result { + let stats = self .inputs .iter() - .map(|input| input.statistics_by_partition()) + .map(|input_exec| input_exec.partition_statistics(partition)) .collect::>>()?; - let all_stats: Vec<_> = input_stats_vec - .iter() - .flat_map(|input_stats| input_stats.iter().map(|stat| Arc::new(stat.clone()))) - .collect(); - - Ok(PartitionedStatistics::new(all_stats)) + Ok(stats + .into_iter() + .reduce(stats_union) + .unwrap_or_else(|| Statistics::new_unknown(&self.schema()))) } fn benefits_from_input_partitioning(&self) -> Vec { 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 36dc2a3799c8d..13b25cb37873a 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -63,7 +63,6 @@ use datafusion_physical_expr::window::{ use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; -use crate::statistics::PartitionedStatistics; use futures::stream::Stream; use futures::{ready, StreamExt}; use hashbrown::hash_table::HashTable; @@ -365,16 +364,9 @@ impl ExecutionPlan for BoundedWindowAggExec { self.statistics_helper(input_stat) } - fn statistics_by_partition(&self) -> Result { - let input_stats = self.input.statistics_by_partition()?; - let stats: Result>> = input_stats - .iter() - .map(|stat| { - let stat = self.statistics_helper(stat.clone())?; - Ok(Arc::new(stat)) - }) - .collect(); - Ok(PartitionedStatistics::new(stats?)) + fn partition_statistics(&self, partition: Option) -> Result { + let input_stat = self.input.partition_statistics(partition)?; + self.statistics_helper(input_stat) } } From 29d0b9981d42a4696890258af9052a185b616774 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 28 Apr 2025 14:15:47 +0800 Subject: [PATCH 23/26] address comments --- datafusion/datasource/src/file_groups.rs | 11 +++++--- datafusion/datasource/src/source.rs | 2 +- datafusion/datasource/src/statistics.rs | 2 +- .../physical-plan/src/execution_plan.rs | 27 +++++++------------ 4 files changed, 20 insertions(+), 22 deletions(-) diff --git a/datafusion/datasource/src/file_groups.rs b/datafusion/datasource/src/file_groups.rs index cb628147d088e..bc9f5f1b78f75 100644 --- a/datafusion/datasource/src/file_groups.rs +++ b/datafusion/datasource/src/file_groups.rs @@ -420,9 +420,14 @@ impl FileGroup { self.files.push(file); } - /// Get the statistics for this group - pub fn statistics_ref(&self) -> Option<&Statistics> { - self.statistics.as_deref() + /// Get the specific file statistics for the given index + /// If the index is None, return the `FileGroup` statistics + pub fn file_statistics(&self, index: Option) -> Option<&Statistics> { + if let Some(index) = index { + self.files.get(index).and_then(|f| f.statistics.as_deref()) + } else { + self.statistics.as_deref() + } } /// Get the mutable reference to the statistics for this group diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index a9ada9ff4de47..022f77f2e4214 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -208,7 +208,7 @@ impl ExecutionPlan for DataSourceExec { self.data_source.as_any().downcast_ref::() { if let Some(file_group) = file_config.file_groups.get(partition) { - if let Some(stat) = file_group.statistics_ref() { + if let Some(stat) = file_group.file_statistics(None) { statistics = stat.clone(); } } diff --git a/datafusion/datasource/src/statistics.rs b/datafusion/datasource/src/statistics.rs index dd9cd87e41406..48aa9fe32ee8e 100644 --- a/datafusion/datasource/src/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -476,7 +476,7 @@ pub fn compute_all_files_statistics( // Then summary statistics across all file groups let file_groups_statistics = file_groups_with_stats .iter() - .filter_map(|file_group| file_group.statistics_ref()); + .filter_map(|file_group| file_group.file_statistics(None)); let mut statistics = Statistics::try_merge_iter(file_groups_statistics, &table_schema)?; diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index f26fb788cae64..9096253b6f381 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -435,25 +435,18 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// (the default), not an error. /// If `partition` is `None`, it returns statistics for the entire plan. fn partition_statistics(&self, partition: Option) -> Result { - match partition { - Some(idx) => { - // Validate partition index - let partition_count = self.properties().partitioning.partition_count(); - if idx >= partition_count { - return internal_err!( - "Invalid partition index: {}, the partition count is {}", - idx, - partition_count - ); - } - // Default implementation: return unknown statistics for the specific partition - Ok(Statistics::new_unknown(&self.schema())) - } - None => { - // Return unknown statistics for the entire plan - Ok(Statistics::new_unknown(&self.schema())) + if let Some(idx) = partition { + // Validate partition index + let partition_count = self.properties().partitioning.partition_count(); + if idx >= partition_count { + return internal_err!( + "Invalid partition index: {}, the partition count is {}", + idx, + partition_count + ); } } + Ok(Statistics::new_unknown(&self.schema())) } /// Returns `true` if a limit can be safely pushed down through this From 90b8f156c25cccbfe57e8d6f50bb25037ff7bf4a Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 28 Apr 2025 14:41:05 +0800 Subject: [PATCH 24/26] deprecated statistics API --- .../core/src/datasource/file_format/csv.rs | 7 +- .../core/src/datasource/file_format/json.rs | 7 +- .../src/datasource/file_format/parquet.rs | 20 +++- .../core/src/datasource/listing/table.rs | 27 ++++- .../core/tests/custom_sources_cases/mod.rs | 7 ++ .../tests/custom_sources_cases/statistics.rs | 18 ++- .../core/tests/parquet/file_statistics.rs | 31 +++-- .../enforce_distribution.rs | 2 +- .../physical_optimizer/join_selection.rs | 80 +++++++++++-- datafusion/core/tests/sql/path_partition.rs | 4 +- datafusion/datasource/src/memory.rs | 2 +- .../src/aggregate_statistics.rs | 2 +- .../src/enforce_distribution.rs | 3 +- .../physical-optimizer/src/join_selection.rs | 6 +- .../src/output_requirements.rs | 6 +- .../physical-plan/src/aggregates/mod.rs | 112 +++++++++++------- .../physical-plan/src/coalesce_batches.rs | 2 +- .../physical-plan/src/coalesce_partitions.rs | 10 +- datafusion/physical-plan/src/display.rs | 11 +- datafusion/physical-plan/src/empty.rs | 7 ++ .../physical-plan/src/execution_plan.rs | 9 ++ datafusion/physical-plan/src/filter.rs | 38 +++--- .../physical-plan/src/joins/cross_join.rs | 5 +- .../physical-plan/src/joins/hash_join.rs | 11 +- .../src/joins/nested_loop_join.rs | 11 +- .../src/joins/sort_merge_join.rs | 11 +- datafusion/physical-plan/src/limit.rs | 22 +--- .../physical-plan/src/placeholder_row.rs | 7 ++ datafusion/physical-plan/src/projection.rs | 6 +- .../physical-plan/src/repartition/mod.rs | 10 +- .../physical-plan/src/sorts/partial_sort.rs | 6 +- datafusion/physical-plan/src/sorts/sort.rs | 2 +- .../src/sorts/sort_preserving_merge.rs | 2 +- datafusion/physical-plan/src/test.rs | 8 ++ datafusion/physical-plan/src/test/exec.rs | 22 ++++ datafusion/physical-plan/src/union.rs | 20 ++-- datafusion/physical-plan/src/values.rs | 4 +- .../src/windows/bounded_window_agg_exec.rs | 3 +- .../src/windows/window_agg_exec.rs | 40 ++++--- datafusion/physical-plan/src/work_table.rs | 4 + 40 files changed, 422 insertions(+), 183 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 323bc28057d43..0c26d7bdb1ee8 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -217,8 +217,11 @@ mod tests { assert_eq!(tt_batches, 50 /* 100/2 */); // test metadata - assert_eq!(exec.statistics()?.num_rows, Precision::Absent); - assert_eq!(exec.statistics()?.total_byte_size, Precision::Absent); + assert_eq!(exec.partition_statistics(None)?.num_rows, Precision::Absent); + assert_eq!( + exec.partition_statistics(None)?.total_byte_size, + Precision::Absent + ); Ok(()) } diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index a70a0f51d3307..48c604efac5c5 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -75,8 +75,11 @@ mod tests { assert_eq!(tt_batches, 6 /* 12/2 */); // test metadata - assert_eq!(exec.statistics()?.num_rows, Precision::Absent); - assert_eq!(exec.statistics()?.total_byte_size, Precision::Absent); + assert_eq!(exec.partition_statistics(None)?.num_rows, Precision::Absent); + assert_eq!( + exec.partition_statistics(None)?.total_byte_size, + Precision::Absent + ); Ok(()) } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 7b8b99273f4ea..0801ae6d8eb34 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -616,9 +616,15 @@ mod tests { assert_eq!(tt_batches, 4 /* 8/2 */); // test metadata - assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8)); + assert_eq!( + exec.partition_statistics(None)?.num_rows, + Precision::Exact(8) + ); // TODO correct byte size: https://github.com/apache/datafusion/issues/14936 - assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671)); + assert_eq!( + exec.partition_statistics(None)?.total_byte_size, + Precision::Exact(671) + ); Ok(()) } @@ -659,9 +665,15 @@ mod tests { get_exec(&state, "alltypes_plain.parquet", projection, Some(1)).await?; // note: even if the limit is set, the executor rounds up to the batch size - assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8)); + assert_eq!( + exec.partition_statistics(None)?.num_rows, + Precision::Exact(8) + ); // TODO correct byte size: https://github.com/apache/datafusion/issues/14936 - assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671)); + assert_eq!( + exec.partition_statistics(None)?.total_byte_size, + Precision::Exact(671) + ); let batches = collect(exec, task_ctx).await?; assert_eq!(1, batches.len()); assert_eq!(11, batches[0].num_columns()); diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index b3e51e6ecd295..e8ef950dfbef0 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1324,8 +1324,14 @@ mod tests { assert_eq!(exec.output_partitioning().partition_count(), 1); // test metadata - assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8)); - assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671)); + assert_eq!( + exec.partition_statistics(None)?.num_rows, + Precision::Exact(8) + ); + assert_eq!( + exec.partition_statistics(None)?.total_byte_size, + Precision::Exact(671) + ); Ok(()) } @@ -1350,9 +1356,15 @@ mod tests { let table = ListingTable::try_new(config)?; let exec = table.scan(&state, None, &[], None).await?; - assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8)); + assert_eq!( + exec.partition_statistics(None)?.num_rows, + Precision::Exact(8) + ); // TODO correct byte size: https://github.com/apache/datafusion/issues/14936 - assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671)); + assert_eq!( + exec.partition_statistics(None)?.total_byte_size, + Precision::Exact(671) + ); Ok(()) } @@ -1378,8 +1390,11 @@ mod tests { let table = ListingTable::try_new(config)?; let exec = table.scan(&state, None, &[], None).await?; - assert_eq!(exec.statistics()?.num_rows, Precision::Absent); - assert_eq!(exec.statistics()?.total_byte_size, Precision::Absent); + assert_eq!(exec.partition_statistics(None)?.num_rows, Precision::Absent); + assert_eq!( + exec.partition_statistics(None)?.total_byte_size, + Precision::Absent + ); Ok(()) } diff --git a/datafusion/core/tests/custom_sources_cases/mod.rs b/datafusion/core/tests/custom_sources_cases/mod.rs index eb930b9a60bc9..cbdc4a448ea41 100644 --- a/datafusion/core/tests/custom_sources_cases/mod.rs +++ b/datafusion/core/tests/custom_sources_cases/mod.rs @@ -180,6 +180,13 @@ impl ExecutionPlan for CustomExecutionPlan { } fn statistics(&self) -> Result { + self.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + return Ok(Statistics::new_unknown(&self.schema())); + } let batch = TEST_CUSTOM_RECORD_BATCH!().unwrap(); Ok(Statistics { num_rows: Precision::Exact(batch.num_rows()), diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index 66c886510e96b..f9b0db0e808c0 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -184,6 +184,14 @@ impl ExecutionPlan for StatisticsValidation { fn statistics(&self) -> Result { Ok(self.stats.clone()) } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + Ok(Statistics::new_unknown(&self.schema)) + } else { + Ok(self.stats.clone()) + } + } } fn init_ctx(stats: Statistics, schema: Schema) -> Result { @@ -232,7 +240,7 @@ async fn sql_basic() -> Result<()> { let physical_plan = df.create_physical_plan().await.unwrap(); // the statistics should be those of the source - assert_eq!(stats, physical_plan.statistics()?); + assert_eq!(stats, physical_plan.partition_statistics(None)?); Ok(()) } @@ -248,7 +256,7 @@ async fn sql_filter() -> Result<()> { .unwrap(); let physical_plan = df.create_physical_plan().await.unwrap(); - let stats = physical_plan.statistics()?; + let stats = physical_plan.partition_statistics(None)?; assert_eq!(stats.num_rows, Precision::Inexact(1)); Ok(()) @@ -270,7 +278,7 @@ async fn sql_limit() -> Result<()> { column_statistics: col_stats, total_byte_size: Precision::Absent }, - physical_plan.statistics()? + physical_plan.partition_statistics(None)? ); let df = ctx @@ -279,7 +287,7 @@ async fn sql_limit() -> Result<()> { .unwrap(); let physical_plan = df.create_physical_plan().await.unwrap(); // when the limit is larger than the original number of lines, statistics remain unchanged - assert_eq!(stats, physical_plan.statistics()?); + assert_eq!(stats, physical_plan.partition_statistics(None)?); Ok(()) } @@ -296,7 +304,7 @@ async fn sql_window() -> Result<()> { let physical_plan = df.create_physical_plan().await.unwrap(); - let result = physical_plan.statistics()?; + let result = physical_plan.partition_statistics(None)?; assert_eq!(stats.num_rows, result.num_rows); let col_stats = result.column_statistics; diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 7e98ebed6c9a7..8b87d59d8c469 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -50,13 +50,19 @@ async fn check_stats_precision_with_filter_pushdown() { let (_, _, state) = get_cache_runtime_state(); // Scan without filter, stats are exact let exec = table.scan(&state, None, &[], None).await.unwrap(); - assert_eq!(exec.statistics().unwrap().num_rows, Precision::Exact(8)); + assert_eq!( + exec.partition_statistics(None).unwrap().num_rows, + Precision::Exact(8) + ); // Scan with filter pushdown, stats are inexact let filter = Expr::gt(col("id"), lit(1)); let exec = table.scan(&state, None, &[filter], None).await.unwrap(); - assert_eq!(exec.statistics().unwrap().num_rows, Precision::Inexact(8)); + assert_eq!( + exec.partition_statistics(None).unwrap().num_rows, + Precision::Inexact(8) + ); } #[tokio::test] @@ -79,9 +85,12 @@ async fn load_table_stats_with_session_level_cache() { assert_eq!(get_static_cache_size(&state1), 0); let exec1 = table1.scan(&state1, None, &[], None).await.unwrap(); - assert_eq!(exec1.statistics().unwrap().num_rows, Precision::Exact(8)); assert_eq!( - exec1.statistics().unwrap().total_byte_size, + exec1.partition_statistics(None).unwrap().num_rows, + Precision::Exact(8) + ); + assert_eq!( + exec1.partition_statistics(None).unwrap().total_byte_size, // TODO correct byte size: https://github.com/apache/datafusion/issues/14936 Precision::Exact(671), ); @@ -91,9 +100,12 @@ async fn load_table_stats_with_session_level_cache() { //check session 1 cache result not show in session 2 assert_eq!(get_static_cache_size(&state2), 0); let exec2 = table2.scan(&state2, None, &[], None).await.unwrap(); - assert_eq!(exec2.statistics().unwrap().num_rows, Precision::Exact(8)); assert_eq!( - exec2.statistics().unwrap().total_byte_size, + exec2.partition_statistics(None).unwrap().num_rows, + Precision::Exact(8) + ); + assert_eq!( + exec2.partition_statistics(None).unwrap().total_byte_size, // TODO correct byte size: https://github.com/apache/datafusion/issues/14936 Precision::Exact(671), ); @@ -103,9 +115,12 @@ async fn load_table_stats_with_session_level_cache() { //check session 1 cache result not show in session 2 assert_eq!(get_static_cache_size(&state1), 1); let exec3 = table1.scan(&state1, None, &[], None).await.unwrap(); - assert_eq!(exec3.statistics().unwrap().num_rows, Precision::Exact(8)); assert_eq!( - exec3.statistics().unwrap().total_byte_size, + exec3.partition_statistics(None).unwrap().num_rows, + Precision::Exact(8) + ); + assert_eq!( + exec3.partition_statistics(None).unwrap().total_byte_size, // TODO correct byte size: https://github.com/apache/datafusion/issues/14936 Precision::Exact(671), ); diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 5e182cb93b39c..bc6957ff7b425 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -170,7 +170,7 @@ impl ExecutionPlan for SortRequiredExec { } fn statistics(&self) -> Result { - self.input.statistics() + self.input.partition_statistics(None) } } diff --git a/datafusion/core/tests/physical_optimizer/join_selection.rs b/datafusion/core/tests/physical_optimizer/join_selection.rs index d3b6ec700beec..d8c0c142f7fb6 100644 --- a/datafusion/core/tests/physical_optimizer/join_selection.rs +++ b/datafusion/core/tests/physical_optimizer/join_selection.rs @@ -251,11 +251,19 @@ async fn test_join_with_swap() { .expect("The type of the plan should not be changed"); assert_eq!( - swapped_join.left().statistics().unwrap().total_byte_size, + swapped_join + .left() + .partition_statistics(None) + .unwrap() + .total_byte_size, Precision::Inexact(8192) ); assert_eq!( - swapped_join.right().statistics().unwrap().total_byte_size, + swapped_join + .right() + .partition_statistics(None) + .unwrap() + .total_byte_size, Precision::Inexact(2097152) ); } @@ -291,11 +299,19 @@ async fn test_left_join_no_swap() { .expect("The type of the plan should not be changed"); assert_eq!( - swapped_join.left().statistics().unwrap().total_byte_size, + swapped_join + .left() + .partition_statistics(None) + .unwrap() + .total_byte_size, Precision::Inexact(8192) ); assert_eq!( - swapped_join.right().statistics().unwrap().total_byte_size, + swapped_join + .right() + .partition_statistics(None) + .unwrap() + .total_byte_size, Precision::Inexact(2097152) ); } @@ -336,11 +352,19 @@ async fn test_join_with_swap_semi() { assert_eq!(swapped_join.schema().fields().len(), 1); assert_eq!( - swapped_join.left().statistics().unwrap().total_byte_size, + swapped_join + .left() + .partition_statistics(None) + .unwrap() + .total_byte_size, Precision::Inexact(8192) ); assert_eq!( - swapped_join.right().statistics().unwrap().total_byte_size, + swapped_join + .right() + .partition_statistics(None) + .unwrap() + .total_byte_size, Precision::Inexact(2097152) ); assert_eq!(original_schema, swapped_join.schema()); @@ -455,11 +479,19 @@ async fn test_join_no_swap() { .expect("The type of the plan should not be changed"); assert_eq!( - swapped_join.left().statistics().unwrap().total_byte_size, + swapped_join + .left() + .partition_statistics(None) + .unwrap() + .total_byte_size, Precision::Inexact(8192) ); assert_eq!( - swapped_join.right().statistics().unwrap().total_byte_size, + swapped_join + .right() + .partition_statistics(None) + .unwrap() + .total_byte_size, Precision::Inexact(2097152) ); } @@ -524,11 +556,19 @@ async fn test_nl_join_with_swap(join_type: JoinType) { ); assert_eq!( - swapped_join.left().statistics().unwrap().total_byte_size, + swapped_join + .left() + .partition_statistics(None) + .unwrap() + .total_byte_size, Precision::Inexact(8192) ); assert_eq!( - swapped_join.right().statistics().unwrap().total_byte_size, + swapped_join + .right() + .partition_statistics(None) + .unwrap() + .total_byte_size, Precision::Inexact(2097152) ); } @@ -589,11 +629,19 @@ async fn test_nl_join_with_swap_no_proj(join_type: JoinType) { ); assert_eq!( - swapped_join.left().statistics().unwrap().total_byte_size, + swapped_join + .left() + .partition_statistics(None) + .unwrap() + .total_byte_size, Precision::Inexact(8192) ); assert_eq!( - swapped_join.right().statistics().unwrap().total_byte_size, + swapped_join + .right() + .partition_statistics(None) + .unwrap() + .total_byte_size, Precision::Inexact(2097152) ); } @@ -1067,6 +1115,14 @@ impl ExecutionPlan for StatisticsExec { fn statistics(&self) -> Result { Ok(self.stats.clone()) } + + fn partition_statistics(&self, partition: Option) -> Result { + Ok(if partition.is_some() { + Statistics::new_unknown(&self.schema) + } else { + self.stats.clone() + }) + } } #[test] diff --git a/datafusion/core/tests/sql/path_partition.rs b/datafusion/core/tests/sql/path_partition.rs index fa6c7432413f1..160084213c7c9 100644 --- a/datafusion/core/tests/sql/path_partition.rs +++ b/datafusion/core/tests/sql/path_partition.rs @@ -511,7 +511,7 @@ async fn parquet_statistics() -> Result<()> { let schema = physical_plan.schema(); assert_eq!(schema.fields().len(), 4); - let stat_cols = physical_plan.statistics()?.column_statistics; + let stat_cols = physical_plan.partition_statistics(None)?.column_statistics; assert_eq!(stat_cols.len(), 4); // stats for the first col are read from the parquet file assert_eq!(stat_cols[0].null_count, Precision::Exact(3)); @@ -526,7 +526,7 @@ async fn parquet_statistics() -> Result<()> { let schema = physical_plan.schema(); assert_eq!(schema.fields().len(), 2); - let stat_cols = physical_plan.statistics()?.column_statistics; + let stat_cols = physical_plan.partition_statistics(None)?.column_statistics; assert_eq!(stat_cols.len(), 2); // stats for the first col are read from the parquet file assert_eq!(stat_cols[0].null_count, Precision::Exact(1)); diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 6d0e16ef4b916..1a62a2a4a41c3 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -976,7 +976,7 @@ mod tests { )?; assert_eq!( - values.statistics()?, + values.partition_statistics(None)?, Statistics { num_rows: Precision::Exact(rows), total_byte_size: Precision::Exact(8), // not important diff --git a/datafusion/physical-optimizer/src/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs index 28ee10eb650a0..6c44c8fe86c5e 100644 --- a/datafusion/physical-optimizer/src/aggregate_statistics.rs +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -53,7 +53,7 @@ impl PhysicalOptimizerRule for AggregateStatistics { .as_any() .downcast_ref::() .expect("take_optimizable() ensures that this is a AggregateExec"); - let stats = partial_agg_exec.input().statistics()?; + let stats = partial_agg_exec.input().partition_statistics(None)?; let mut projections = vec![]; for expr in partial_agg_exec.aggr_expr() { let field = expr.field(); diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 947fd3eba23e6..0c80cce4cc020 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1114,7 +1114,8 @@ fn get_repartition_requirement_status( { // Decide whether adding a round robin is beneficial depending on // the statistical information we have on the number of rows: - let roundrobin_beneficial_stats = match child.statistics()?.num_rows { + let roundrobin_beneficial_stats = match child.partition_statistics(None)?.num_rows + { Precision::Exact(n_rows) => n_rows > batch_size, Precision::Inexact(n_rows) => !should_use_estimates || (n_rows > batch_size), Precision::Absent => true, diff --git a/datafusion/physical-optimizer/src/join_selection.rs b/datafusion/physical-optimizer/src/join_selection.rs index 5a772ccdd249f..05758e5dfdf10 100644 --- a/datafusion/physical-optimizer/src/join_selection.rs +++ b/datafusion/physical-optimizer/src/join_selection.rs @@ -65,8 +65,8 @@ pub(crate) fn should_swap_join_order( // Get the left and right table's total bytes // If both the left and right tables contain total_byte_size statistics, // use `total_byte_size` to determine `should_swap_join_order`, else use `num_rows` - let left_stats = left.statistics()?; - let right_stats = right.statistics()?; + let left_stats = left.partition_statistics(None)?; + let right_stats = right.partition_statistics(None)?; // First compare `total_byte_size` of left and right side, // if information in this field is insufficient fallback to the `num_rows` match ( @@ -91,7 +91,7 @@ fn supports_collect_by_thresholds( ) -> bool { // Currently we do not trust the 0 value from stats, due to stats collection might have bug // TODO check the logic in datasource::get_statistics_with_limit() - let Ok(stats) = plan.statistics() else { + let Ok(stats) = plan.partition_statistics(None) else { return false; }; diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 3ca0547aa11d8..0488b3fd49a86 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -200,7 +200,11 @@ impl ExecutionPlan for OutputRequirementExec { } fn statistics(&self) -> Result { - self.input.statistics() + self.input.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + self.input.partition_statistics(partition) } fn try_swapping_with_projection( diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 8906468f68db2..2a07caceab899 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -735,6 +735,59 @@ impl AggregateExec { pub fn input_order_mode(&self) -> &InputOrderMode { &self.input_order_mode } + + fn statistics(&self) -> Result { + // TODO stats: group expressions: + // - once expressions will be able to compute their own stats, use it here + // - case where we group by on a column for which with have the `distinct` stat + // TODO stats: aggr expression: + // - aggregations sometimes also preserve invariants such as min, max... + let column_statistics = Statistics::unknown_column(&self.schema()); + match self.mode { + AggregateMode::Final | AggregateMode::FinalPartitioned + if self.group_by.expr.is_empty() => + { + Ok(Statistics { + num_rows: Precision::Exact(1), + column_statistics, + total_byte_size: Precision::Absent, + }) + } + _ => { + // When the input row count is 0 or 1, we can adopt that statistic keeping its reliability. + // When it is larger than 1, we degrade the precision since it may decrease after aggregation. + let num_rows = if let Some(value) = self + .input() + .partition_statistics(None)? + .num_rows + .get_value() + { + if *value > 1 { + self.input() + .partition_statistics(None)? + .num_rows + .to_inexact() + } else if *value == 0 { + // Aggregation on an empty table creates a null row. + self.input() + .partition_statistics(None)? + .num_rows + .add(&Precision::Exact(1)) + } else { + // num_rows = 1 case + self.input().partition_statistics(None)?.num_rows + } + } else { + Precision::Absent + }; + Ok(Statistics { + num_rows, + column_statistics, + total_byte_size: Precision::Absent, + }) + } + } + } } impl DisplayAs for AggregateExec { @@ -941,49 +994,15 @@ impl ExecutionPlan for AggregateExec { } fn statistics(&self) -> Result { - // TODO stats: group expressions: - // - once expressions will be able to compute their own stats, use it here - // - case where we group by on a column for which with have the `distinct` stat - // TODO stats: aggr expression: - // - aggregations sometimes also preserve invariants such as min, max... - let column_statistics = Statistics::unknown_column(&self.schema()); - match self.mode { - AggregateMode::Final | AggregateMode::FinalPartitioned - if self.group_by.expr.is_empty() => - { - Ok(Statistics { - num_rows: Precision::Exact(1), - column_statistics, - total_byte_size: Precision::Absent, - }) - } - _ => { - // When the input row count is 0 or 1, we can adopt that statistic keeping its reliability. - // When it is larger than 1, we degrade the precision since it may decrease after aggregation. - let num_rows = if let Some(value) = - self.input().statistics()?.num_rows.get_value() - { - if *value > 1 { - self.input().statistics()?.num_rows.to_inexact() - } else if *value == 0 { - // Aggregation on an empty table creates a null row. - self.input() - .statistics()? - .num_rows - .add(&Precision::Exact(1)) - } else { - // num_rows = 1 case - self.input().statistics()?.num_rows - } - } else { - Precision::Absent - }; - Ok(Statistics { - num_rows, - column_statistics, - total_byte_size: Precision::Absent, - }) - } + self.statistics() + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_none() { + // If the partition is not specified, we can use the statistics of the input plan + self.statistics() + } else { + Ok(Statistics::new_unknown(&self.schema())) } } @@ -1924,6 +1943,13 @@ mod tests { } fn statistics(&self) -> Result { + self.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + return Ok(Statistics::new_unknown(self.schema().as_ref())); + } let (_, batches) = some_data(); Ok(common::compute_record_batch_statistics( &[batches], diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 09f06fe0f04b7..06483e4a51148 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -196,7 +196,7 @@ impl ExecutionPlan for CoalesceBatchesExec { } fn statistics(&self) -> Result { - Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) + self.partition_statistics(None) } fn partition_statistics(&self, partition: Option) -> Result { diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 9f210c3a2abc6..3cdc61d40eaf9 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -196,11 +196,17 @@ impl ExecutionPlan for CoalescePartitionsExec { } fn statistics(&self) -> Result { - Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) + self.partition_statistics(None) } fn partition_statistics(&self, _partition: Option) -> Result { - Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) + Statistics::with_fetch( + self.input.partition_statistics(None)?, + self.schema(), + self.fetch, + 0, + 1, + ) } fn supports_limit_pushdown(&self) -> bool { diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index e247f5ad9d194..5a44f8735c8f4 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -394,7 +394,7 @@ impl ExecutionPlanVisitor for IndentVisitor<'_, '_> { } } if self.show_statistics { - let stats = plan.statistics().map_err(|_e| fmt::Error)?; + let stats = plan.partition_statistics(None).map_err(|_e| fmt::Error)?; write!(self.f, ", statistics=[{}]", stats)?; } if self.show_schema { @@ -479,7 +479,7 @@ impl ExecutionPlanVisitor for GraphvizVisitor<'_, '_> { }; let statistics = if self.show_statistics { - let stats = plan.statistics().map_err(|_e| fmt::Error)?; + let stats = plan.partition_statistics(None).map_err(|_e| fmt::Error)?; format!("statistics=[{}]", stats) } else { "".to_string() @@ -1120,6 +1120,13 @@ mod tests { } fn statistics(&self) -> Result { + self.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + return Ok(Statistics::new_unknown(self.schema().as_ref())); + } match self { Self::Panic => panic!("expected panic"), Self::Error => { diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index 3fdde39df6f11..6c8133de4624f 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -150,6 +150,13 @@ impl ExecutionPlan for EmptyExec { } fn statistics(&self) -> Result { + self.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + return Ok(Statistics::new_unknown(&self.schema())); + } let batch = self .data() .expect("Create empty RecordBatch should not fail"); diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 9096253b6f381..9551c2b1743e1 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -426,6 +426,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// /// For TableScan executors, which supports filter pushdown, special attention /// needs to be paid to whether the stats returned by this method are exact or not + #[deprecated(since = "48.0.0", note = "Use `partition_statistics` method instead")] fn statistics(&self) -> Result { Ok(Statistics::new_unknown(&self.schema())) } @@ -1196,6 +1197,10 @@ mod tests { fn statistics(&self) -> Result { unimplemented!() } + + fn partition_statistics(&self, _partition: Option) -> Result { + unimplemented!() + } } #[derive(Debug)] @@ -1259,6 +1264,10 @@ mod tests { fn statistics(&self) -> Result { unimplemented!() } + + fn partition_statistics(&self, _partition: Option) -> Result { + unimplemented!() + } } #[test] diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 848bb0e802cf0..6df3e236a0ddc 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -261,7 +261,7 @@ impl FilterExec { // to construct the equivalence properties: let stats = Self::statistics_helper( input.schema(), - input.statistics()?, + input.partition_statistics(None)?, predicate, default_selectivity, )?; @@ -404,13 +404,7 @@ impl ExecutionPlan for FilterExec { /// The output statistics of a filtering operation can be estimated if the /// predicate's selectivity value can be determined for the incoming data. fn statistics(&self) -> Result { - let stats = Self::statistics_helper( - self.schema(), - self.input().statistics()?, - self.predicate(), - self.default_selectivity, - )?; - Ok(stats.project(self.projection.as_ref())) + self.partition_statistics(None) } fn partition_statistics(&self, partition: Option) -> Result { @@ -773,7 +767,7 @@ mod tests { let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics()?; + let statistics = filter.partition_statistics(None)?; assert_eq!(statistics.num_rows, Precision::Inexact(25)); assert_eq!( statistics.total_byte_size, @@ -823,7 +817,7 @@ mod tests { sub_filter, )?); - let statistics = filter.statistics()?; + let statistics = filter.partition_statistics(None)?; assert_eq!(statistics.num_rows, Precision::Inexact(16)); assert_eq!( statistics.column_statistics, @@ -883,7 +877,7 @@ mod tests { binary(col("a", &schema)?, Operator::GtEq, lit(10i32), &schema)?, b_gt_5, )?); - let statistics = filter.statistics()?; + let statistics = filter.partition_statistics(None)?; // On a uniform distribution, only fifteen rows will satisfy the // filter that 'a' proposed (a >= 10 AND a <= 25) (15/100) and only // 5 rows will satisfy the filter that 'b' proposed (b > 45) (5/50). @@ -928,7 +922,7 @@ mod tests { let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics()?; + let statistics = filter.partition_statistics(None)?; assert_eq!(statistics.num_rows, Precision::Absent); Ok(()) @@ -1001,7 +995,7 @@ mod tests { )); let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics()?; + let statistics = filter.partition_statistics(None)?; // 0.5 (from a) * 0.333333... (from b) * 0.798387... (from c) ≈ 0.1330... // num_rows after ceil => 133.0... => 134 // total_byte_size after ceil => 532.0... => 533 @@ -1097,10 +1091,10 @@ mod tests { )), )); // Since filter predicate passes all entries, statistics after filter shouldn't change. - let expected = input.statistics()?.column_statistics; + let expected = input.partition_statistics(None)?.column_statistics; let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics()?; + let statistics = filter.partition_statistics(None)?; assert_eq!(statistics.num_rows, Precision::Inexact(1000)); assert_eq!(statistics.total_byte_size, Precision::Inexact(4000)); @@ -1153,7 +1147,7 @@ mod tests { )); let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics()?; + let statistics = filter.partition_statistics(None)?; assert_eq!(statistics.num_rows, Precision::Inexact(0)); assert_eq!(statistics.total_byte_size, Precision::Inexact(0)); @@ -1213,7 +1207,7 @@ mod tests { )); let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics()?; + let statistics = filter.partition_statistics(None)?; assert_eq!(statistics.num_rows, Precision::Inexact(490)); assert_eq!(statistics.total_byte_size, Precision::Inexact(1960)); @@ -1263,7 +1257,7 @@ mod tests { )); let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let filter_statistics = filter.statistics()?; + let filter_statistics = filter.partition_statistics(None)?; let expected_filter_statistics = Statistics { num_rows: Precision::Absent, @@ -1297,7 +1291,7 @@ mod tests { )); let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let filter_statistics = filter.statistics()?; + let filter_statistics = filter.partition_statistics(None)?; // First column is "a", and it is a column with only one value after the filter. assert!(filter_statistics.column_statistics[0].is_singleton()); @@ -1344,11 +1338,11 @@ mod tests { Arc::new(Literal::new(ScalarValue::Decimal128(Some(10), 10, 10))), )); let filter = FilterExec::try_new(predicate, input)?; - let statistics = filter.statistics()?; + let statistics = filter.partition_statistics(None)?; assert_eq!(statistics.num_rows, Precision::Inexact(200)); assert_eq!(statistics.total_byte_size, Precision::Inexact(800)); let filter = filter.with_default_selectivity(40)?; - let statistics = filter.statistics()?; + let statistics = filter.partition_statistics(None)?; assert_eq!(statistics.num_rows, Precision::Inexact(400)); assert_eq!(statistics.total_byte_size, Precision::Inexact(1600)); Ok(()) @@ -1382,7 +1376,7 @@ mod tests { Arc::new(EmptyExec::new(Arc::clone(&schema))), )?; - exec.statistics().unwrap(); + exec.partition_statistics(None).unwrap(); Ok(()) } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index fb0de0f3a1a59..cbc38f1149725 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -337,10 +337,7 @@ impl ExecutionPlan for CrossJoinExec { } fn statistics(&self) -> Result { - Ok(stats_cartesian_product( - self.left.statistics()?, - self.right.statistics()?, - )) + self.partition_statistics(None) } fn partition_statistics(&self, partition: Option) -> Result { diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 50282e056777d..dd77c5e37c3ef 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -879,12 +879,19 @@ impl ExecutionPlan for HashJoinExec { } fn statistics(&self) -> Result { + self.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + return Ok(Statistics::new_unknown(&self.schema())); + } // TODO stats: it is not possible in general to know the output size of joins // There are some special cases though, for example: // - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)` let stats = estimate_join_statistics( - self.left.statistics()?, - self.right.statistics()?, + self.left.partition_statistics(None)?, + self.right.partition_statistics(None)?, self.on.clone(), &self.join_type, &self.join_schema, diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 94b6725bb19cf..f913bcaa947b5 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -567,9 +567,16 @@ impl ExecutionPlan for NestedLoopJoinExec { } fn statistics(&self) -> Result { + self.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + return Ok(Statistics::new_unknown(&self.schema())); + } estimate_join_statistics( - self.left.statistics()?, - self.right.statistics()?, + self.left.partition_statistics(None)?, + self.right.partition_statistics(None)?, vec![], &self.join_type, &self.join_schema, diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 28612540bce5b..6817bd9b76dd4 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -514,12 +514,19 @@ impl ExecutionPlan for SortMergeJoinExec { } fn statistics(&self) -> Result { + self.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + return Ok(Statistics::new_unknown(&self.schema())); + } // TODO stats: it is not possible in general to know the output size of joins // There are some special cases though, for example: // - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)` estimate_join_statistics( - self.left.statistics()?, - self.right.statistics()?, + self.left.partition_statistics(None)?, + self.right.partition_statistics(None)?, self.on.clone(), &self.join_type, &self.schema, diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 3d14eb41584bb..87fe4df4717e8 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -193,13 +193,7 @@ impl ExecutionPlan for GlobalLimitExec { } fn statistics(&self) -> Result { - Statistics::with_fetch( - self.input.statistics()?, - self.schema(), - self.fetch, - self.skip, - 1, - ) + self.partition_statistics(None) } fn partition_statistics(&self, partition: Option) -> Result { @@ -339,13 +333,7 @@ impl ExecutionPlan for LocalLimitExec { } fn statistics(&self) -> Result { - Statistics::with_fetch( - self.input.statistics()?, - self.schema(), - Some(self.fetch), - 0, - 1, - ) + self.partition_statistics(None) } fn partition_statistics(&self, partition: Option) -> Result { @@ -775,7 +763,7 @@ mod tests { let offset = GlobalLimitExec::new(Arc::new(CoalescePartitionsExec::new(csv)), skip, fetch); - Ok(offset.statistics()?.num_rows) + Ok(offset.partition_statistics(None)?.num_rows) } pub fn build_group_by( @@ -815,7 +803,7 @@ mod tests { fetch, ); - Ok(offset.statistics()?.num_rows) + Ok(offset.partition_statistics(None)?.num_rows) } async fn row_number_statistics_for_local_limit( @@ -828,7 +816,7 @@ mod tests { let offset = LocalLimitExec::new(csv, fetch); - Ok(offset.statistics()?.num_rows) + Ok(offset.partition_statistics(None)?.num_rows) } /// Return a RecordBatch with a single array with row_count sz diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index eecd980d09f8a..46847b2413c0e 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -166,6 +166,13 @@ impl ExecutionPlan for PlaceholderRowExec { } fn statistics(&self) -> Result { + self.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + return Ok(Statistics::new_unknown(&self.schema())); + } let batch = self .data() .expect("Create single row placeholder RecordBatch should not fail"); diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 6db545f519b78..8761d64b60b5c 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -244,11 +244,7 @@ impl ExecutionPlan for ProjectionExec { } fn statistics(&self) -> Result { - Ok(stats_projection( - self.input.statistics()?, - self.expr.iter().map(|(e, _)| Arc::clone(e)), - Arc::clone(&self.schema), - )) + self.partition_statistics(None) } fn partition_statistics(&self, partition: Option) -> Result { diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index c480fc2abaa1a..f7c4f7477f121 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -688,7 +688,15 @@ impl ExecutionPlan for RepartitionExec { } fn statistics(&self) -> Result { - self.input.statistics() + self.input.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_none() { + self.input.partition_statistics(None) + } else { + Ok(Statistics::new_unknown(&self.schema())) + } } fn cardinality_effect(&self) -> CardinalityEffect { diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 320fa21c86656..6d7f64ebfbd52 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -321,7 +321,11 @@ impl ExecutionPlan for PartialSortExec { } fn statistics(&self) -> Result { - self.input.statistics() + self.input.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + self.input.partition_statistics(partition) } } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index facdd1eea94ed..93195fbfcbfdc 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1297,7 +1297,7 @@ impl ExecutionPlan for SortExec { } fn statistics(&self) -> Result { - Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) + self.partition_statistics(None) } fn partition_statistics(&self, partition: Option) -> Result { diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 4ca522b267d45..da346d8f8a4e5 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -343,7 +343,7 @@ impl ExecutionPlan for SortPreservingMergeExec { } fn statistics(&self) -> Result { - self.input.statistics() + self.input.partition_statistics(None) } fn partition_statistics(&self, _partition: Option) -> Result { diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index a2dc1d778436a..52b31819ea723 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -173,6 +173,14 @@ impl ExecutionPlan for TestMemoryExec { self.statistics() } + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + Ok(Statistics::new_unknown(&self.schema)) + } else { + self.statistics() + } + } + fn fetch(&self) -> Option { self.fetch } diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index d0a0d25779cc8..12ffca871f073 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -255,6 +255,13 @@ impl ExecutionPlan for MockExec { // Panics if one of the batches is an error fn statistics(&self) -> Result { + self.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + return Ok(Statistics::new_unknown(&self.schema)); + } let data: Result> = self .data .iter() @@ -405,6 +412,13 @@ impl ExecutionPlan for BarrierExec { } fn statistics(&self) -> Result { + self.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + return Ok(Statistics::new_unknown(&self.schema)); + } Ok(common::compute_record_batch_statistics( &self.data, &self.schema, @@ -590,6 +604,14 @@ impl ExecutionPlan for StatisticsExec { fn statistics(&self) -> Result { Ok(self.stats.clone()) } + + fn partition_statistics(&self, partition: Option) -> Result { + Ok(if partition.is_some() { + Statistics::new_unknown(&self.schema) + } else { + self.stats.clone() + }) + } } /// Execution plan that emits streams that block forever. diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 5b9e18bdbd7b8..94687b5874562 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -258,16 +258,7 @@ impl ExecutionPlan for UnionExec { } fn statistics(&self) -> Result { - let stats = self - .inputs - .iter() - .map(|stat| stat.statistics()) - .collect::>>()?; - - Ok(stats - .into_iter() - .reduce(stats_union) - .unwrap_or_else(|| Statistics::new_unknown(&self.schema()))) + self.partition_statistics(None) } fn partition_statistics(&self, partition: Option) -> Result { @@ -484,10 +475,17 @@ impl ExecutionPlan for InterleaveExec { } fn statistics(&self) -> Result { + self.partition_statistics(None) + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_some() { + return Ok(Statistics::new_unknown(&self.schema())); + } let stats = self .inputs .iter() - .map(|stat| stat.statistics()) + .map(|stat| stat.partition_statistics(None)) .collect::>>()?; Ok(stats diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 6cb64bcb5d867..fb27ccf30179a 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -308,8 +308,10 @@ mod tests { data, )?; + #[allow(deprecated)] + let stats = values.statistics()?; assert_eq!( - values.statistics()?, + stats, Statistics { num_rows: Precision::Exact(rows), total_byte_size: Precision::Exact(8), // not important 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 13b25cb37873a..1ea02adafe3f8 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -360,8 +360,7 @@ impl ExecutionPlan for BoundedWindowAggExec { } fn statistics(&self) -> Result { - let input_stat = self.input.statistics()?; - self.statistics_helper(input_stat) + self.partition_statistics(None) } fn partition_statistics(&self, partition: Option) -> Result { diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 3c42d3032ed5d..e780d8aa2857c 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -156,6 +156,24 @@ impl WindowAggExec { .unwrap_or_else(Vec::new) } } + + fn statistics(&self) -> Result { + let input_stat = self.input.partition_statistics(None)?; + let win_cols = self.window_expr.len(); + let input_cols = self.input.schema().fields().len(); + // TODO stats: some windowing function will maintain invariants such as min, max... + let mut column_statistics = Vec::with_capacity(win_cols + input_cols); + // copy stats of the input to the beginning of the schema. + column_statistics.extend(input_stat.column_statistics); + for _ in 0..win_cols { + column_statistics.push(ColumnStatistics::new_unknown()) + } + Ok(Statistics { + num_rows: input_stat.num_rows, + column_statistics, + total_byte_size: Precision::Absent, + }) + } } impl DisplayAs for WindowAggExec { @@ -271,21 +289,15 @@ impl ExecutionPlan for WindowAggExec { } fn statistics(&self) -> Result { - let input_stat = self.input.statistics()?; - let win_cols = self.window_expr.len(); - let input_cols = self.input.schema().fields().len(); - // TODO stats: some windowing function will maintain invariants such as min, max... - let mut column_statistics = Vec::with_capacity(win_cols + input_cols); - // copy stats of the input to the beginning of the schema. - column_statistics.extend(input_stat.column_statistics); - for _ in 0..win_cols { - column_statistics.push(ColumnStatistics::new_unknown()) + self.statistics() + } + + fn partition_statistics(&self, partition: Option) -> Result { + if partition.is_none() { + self.statistics() + } else { + Ok(Statistics::new_unknown(&self.schema())) } - Ok(Statistics { - num_rows: input_stat.num_rows, - column_statistics, - total_byte_size: Precision::Absent, - }) } } diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index 126a7d0bba294..eea1b99586330 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -227,6 +227,10 @@ impl ExecutionPlan for WorkTableExec { fn statistics(&self) -> Result { Ok(Statistics::new_unknown(&self.schema())) } + + fn partition_statistics(&self, _partition: Option) -> Result { + Ok(Statistics::new_unknown(&self.schema())) + } } #[cfg(test)] From 71777d454a9abcd87a91f14949096261359dbdec Mon Sep 17 00:00:00 2001 From: xudong963 Date: Tue, 29 Apr 2025 16:08:57 +0800 Subject: [PATCH 25/26] rebase main and fix tests --- .../core/src/datasource/listing/table.rs | 15 ++++++-- .../partition_statistics.rs | 5 +-- datafusion/physical-plan/src/union.rs | 34 ++++++++++++++----- 3 files changed, 40 insertions(+), 14 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index e8ef950dfbef0..e9116ffca9eda 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1154,9 +1154,18 @@ impl ListingTable { .map_schema(self.file_schema.as_ref())?; // Use schema_mapper to map each file-level column statistics to table-level column statistics file_groups.iter_mut().try_for_each(|file_group| { - if let Some(stat) = file_group.statistics_mut() { - stat.column_statistics = - schema_mapper.map_column_statistics(&stat.column_statistics)?; + // Update each file's statistics's column statistics in file_group + for idx in 0..file_group.len() { + if let Some(stat) = file_group[idx].statistics.as_ref() { + let column_statistics = + schema_mapper.map_column_statistics(&stat.column_statistics)?; + // Update the file's statistics with the mapped column statistics + file_group[idx].statistics = Some(Arc::new(Statistics { + num_rows: stat.num_rows.clone(), + total_byte_size: stat.total_byte_size.clone(), + column_statistics, + })); + } } Ok::<_, DataFusionError>(()) })?; diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index b1bd3cc7f19a0..8ac583c7150bb 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -476,14 +476,15 @@ mod test { #[tokio::test] async fn test_statistic_by_partition_of_global_limit_partitions() -> Result<()> { let scan = create_scan_exec_with_statistics(None, Some(2)).await; + // Skip 2 rows let global_limit: Arc = Arc::new(GlobalLimitExec::new(scan.clone(), 0, Some(2))); let statistics = (0..global_limit.output_partitioning().partition_count()) .map(|idx| global_limit.partition_statistics(Some(idx))) .collect::>>()?; assert_eq!(statistics.len(), 1); - let mut expected_statistic_partition = Statistics::new_unknown(&scan.schema()); - expected_statistic_partition.num_rows = Precision::Exact(2); + let expected_statistic_partition = + create_partition_statistics(2, 110, 3, 4, true); assert_eq!(statistics[0], expected_statistic_partition); Ok(()) } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 94687b5874562..e5b53f8e8e910 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -262,16 +262,32 @@ impl ExecutionPlan for UnionExec { } fn partition_statistics(&self, partition: Option) -> Result { - let stats = self - .inputs - .iter() - .map(|input_exec| input_exec.partition_statistics(partition)) - .collect::>>()?; + if let Some(partition_idx) = partition { + // For a specific partition, find which input it belongs to + let mut remaining_idx = partition_idx; + for input in &self.inputs { + let input_partition_count = input.output_partitioning().partition_count(); + if remaining_idx < input_partition_count { + // This partition belongs to this input + return input.partition_statistics(Some(remaining_idx)); + } + remaining_idx -= input_partition_count; + } + // If we get here, the partition index is out of bounds + return Ok(Statistics::new_unknown(&self.schema())); + } else { + // Collect statistics from all inputs + let stats = self + .inputs + .iter() + .map(|input_exec| input_exec.partition_statistics(None)) + .collect::>>()?; - Ok(stats - .into_iter() - .reduce(stats_union) - .unwrap_or_else(|| Statistics::new_unknown(&self.schema()))) + Ok(stats + .into_iter() + .reduce(stats_union) + .unwrap_or_else(|| Statistics::new_unknown(&self.schema()))) + } } fn benefits_from_input_partitioning(&self) -> Vec { From 689fc669c47581b86d6e4c12d73210f997c4cb10 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Tue, 29 Apr 2025 16:55:58 +0800 Subject: [PATCH 26/26] fix --- .../core/src/datasource/listing/table.rs | 33 ++++++++----------- .../physical-plan/src/aggregates/mod.rs | 6 ++-- .../physical-plan/src/coalesce_batches.rs | 10 +++--- .../physical-plan/src/coalesce_partitions.rs | 10 ++---- datafusion/physical-plan/src/limit.rs | 16 ++++++--- datafusion/physical-plan/src/sorts/sort.rs | 11 ++++--- datafusion/physical-plan/src/test.rs | 6 ++-- datafusion/physical-plan/src/union.rs | 2 +- .../src/windows/window_agg_exec.rs | 6 ++-- .../test_files/listing_table_statistics.slt | 2 +- 10 files changed, 52 insertions(+), 50 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index e9116ffca9eda..84a63faffbbd3 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1149,32 +1149,25 @@ impl ListingTable { let (file_group, inexact_stats) = get_files_with_limit(files, limit, self.options.collect_stat).await?; - let mut file_groups = file_group.split_files(self.options.target_partitions); + let file_groups = file_group.split_files(self.options.target_partitions); + let (mut file_groups, mut stats) = compute_all_files_statistics( + file_groups, + self.schema(), + self.options.collect_stat, + inexact_stats, + )?; let (schema_mapper, _) = DefaultSchemaAdapterFactory::from_schema(self.schema()) .map_schema(self.file_schema.as_ref())?; - // Use schema_mapper to map each file-level column statistics to table-level column statistics + stats.column_statistics = + schema_mapper.map_column_statistics(&stats.column_statistics)?; file_groups.iter_mut().try_for_each(|file_group| { - // Update each file's statistics's column statistics in file_group - for idx in 0..file_group.len() { - if let Some(stat) = file_group[idx].statistics.as_ref() { - let column_statistics = - schema_mapper.map_column_statistics(&stat.column_statistics)?; - // Update the file's statistics with the mapped column statistics - file_group[idx].statistics = Some(Arc::new(Statistics { - num_rows: stat.num_rows.clone(), - total_byte_size: stat.total_byte_size.clone(), - column_statistics, - })); - } + if let Some(stat) = file_group.statistics_mut() { + stat.column_statistics = + schema_mapper.map_column_statistics(&stat.column_statistics)?; } Ok::<_, DataFusionError>(()) })?; - compute_all_files_statistics( - file_groups, - self.schema(), - self.options.collect_stat, - inexact_stats, - ) + Ok((file_groups, stats)) } /// Collects statistics for a given partitioned file. diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2a07caceab899..a6fc2fc682a2a 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -736,7 +736,7 @@ impl AggregateExec { &self.input_order_mode } - fn statistics(&self) -> Result { + fn statistics_inner(&self) -> Result { // TODO stats: group expressions: // - once expressions will be able to compute their own stats, use it here // - case where we group by on a column for which with have the `distinct` stat @@ -994,13 +994,13 @@ impl ExecutionPlan for AggregateExec { } fn statistics(&self) -> Result { - self.statistics() + self.statistics_inner() } fn partition_statistics(&self, partition: Option) -> Result { if partition.is_none() { // If the partition is not specified, we can use the statistics of the input plan - self.statistics() + self.statistics_inner() } else { Ok(Statistics::new_unknown(&self.schema())) } diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 06483e4a51148..34b3f1b0241b3 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -200,10 +200,12 @@ impl ExecutionPlan for CoalesceBatchesExec { } fn partition_statistics(&self, partition: Option) -> Result { - let input_stats = self.input.partition_statistics(partition)?; - let fetched_stat = - Statistics::with_fetch(input_stats.clone(), self.schema(), self.fetch, 0, 1)?; - Ok(fetched_stat) + self.input.partition_statistics(partition)?.with_fetch( + self.schema(), + self.fetch, + 0, + 1, + ) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 3cdc61d40eaf9..114f830688c97 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -200,13 +200,9 @@ impl ExecutionPlan for CoalescePartitionsExec { } fn partition_statistics(&self, _partition: Option) -> Result { - Statistics::with_fetch( - self.input.partition_statistics(None)?, - self.schema(), - self.fetch, - 0, - 1, - ) + self.input + .partition_statistics(None)? + .with_fetch(self.schema(), self.fetch, 0, 1) } fn supports_limit_pushdown(&self) -> bool { diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 87fe4df4717e8..80c393eec485e 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -197,8 +197,12 @@ impl ExecutionPlan for GlobalLimitExec { } fn partition_statistics(&self, partition: Option) -> Result { - let input_stats = self.input.partition_statistics(partition)?; - Statistics::with_fetch(input_stats, self.schema(), self.fetch, self.skip, 1) + self.input.partition_statistics(partition)?.with_fetch( + self.schema(), + self.fetch, + self.skip, + 1, + ) } fn fetch(&self) -> Option { @@ -337,8 +341,12 @@ impl ExecutionPlan for LocalLimitExec { } fn partition_statistics(&self, partition: Option) -> Result { - let input_stats = self.input.partition_statistics(partition)?; - Statistics::with_fetch(input_stats, self.schema(), Some(self.fetch), 0, 1) + self.input.partition_statistics(partition)?.with_fetch( + self.schema(), + Some(self.fetch), + 0, + 1, + ) } fn fetch(&self) -> Option { diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 93195fbfcbfdc..bf7d69b62f215 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1302,16 +1302,19 @@ impl ExecutionPlan for SortExec { fn partition_statistics(&self, partition: Option) -> Result { if !self.preserve_partitioning() { - return Statistics::with_fetch( - self.input.partition_statistics(None)?, + return self.input.partition_statistics(None)?.with_fetch( self.schema(), self.fetch, 0, 1, ); } - let input_stats = self.input.partition_statistics(partition)?; - Statistics::with_fetch(input_stats, self.schema(), self.fetch, 0, 1) + self.input.partition_statistics(partition)?.with_fetch( + self.schema(), + self.fetch, + 0, + 1, + ) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index 52b31819ea723..cb0b060d272ae 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -170,14 +170,14 @@ impl ExecutionPlan for TestMemoryExec { } fn statistics(&self) -> Result { - self.statistics() + self.statistics_inner() } fn partition_statistics(&self, partition: Option) -> Result { if partition.is_some() { Ok(Statistics::new_unknown(&self.schema)) } else { - self.statistics() + self.statistics_inner() } } @@ -222,7 +222,7 @@ impl TestMemoryExec { ) } - fn statistics(&self) -> Result { + fn statistics_inner(&self) -> Result { Ok(common::compute_record_batch_statistics( &self.partitions, &self.schema, diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index e5b53f8e8e910..bdae97f5d13d7 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -274,7 +274,7 @@ impl ExecutionPlan for UnionExec { remaining_idx -= input_partition_count; } // If we get here, the partition index is out of bounds - return Ok(Statistics::new_unknown(&self.schema())); + Ok(Statistics::new_unknown(&self.schema())) } else { // Collect statistics from all inputs let stats = self diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index e780d8aa2857c..4c76e22308759 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -157,7 +157,7 @@ impl WindowAggExec { } } - fn statistics(&self) -> Result { + fn statistics_inner(&self) -> Result { let input_stat = self.input.partition_statistics(None)?; let win_cols = self.window_expr.len(); let input_cols = self.input.schema().fields().len(); @@ -289,12 +289,12 @@ impl ExecutionPlan for WindowAggExec { } fn statistics(&self) -> Result { - self.statistics() + self.statistics_inner() } fn partition_statistics(&self, partition: Option) -> Result { if partition.is_none() { - self.statistics() + self.statistics_inner() } else { Ok(Statistics::new_unknown(&self.schema())) } diff --git a/datafusion/sqllogictest/test_files/listing_table_statistics.slt b/datafusion/sqllogictest/test_files/listing_table_statistics.slt index aeeaaea6c2a3d..8efe5f0d9ae51 100644 --- a/datafusion/sqllogictest/test_files/listing_table_statistics.slt +++ b/datafusion/sqllogictest/test_files/listing_table_statistics.slt @@ -44,4 +44,4 @@ statement ok set datafusion.execution.collect_statistics = false; statement ok -set datafusion.explain.show_statistics = false; \ No newline at end of file +set datafusion.explain.show_statistics = false;