diff --git a/Cargo.lock b/Cargo.lock index b22fc5078976..24efe4617eee 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2447,6 +2447,8 @@ version = "47.0.0" dependencies = [ "arrow", "datafusion-common", + "datafusion-datasource", + "datafusion-datasource-parquet", "datafusion-execution", "datafusion-expr", "datafusion-expr-common", diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index aaadb09bcc98..49eacc591fd7 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -40,6 +40,7 @@ recursive_protection = ["dep:recursive"] [dependencies] arrow = { workspace = true } datafusion-common = { workspace = true, default-features = true } +datafusion-datasource = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-expr-common = { workspace = true, default-features = true } @@ -51,6 +52,7 @@ log = { workspace = true } recursive = { workspace = true, optional = true } [dev-dependencies] +datafusion-datasource-parquet = { workspace = true } datafusion-expr = { workspace = true } datafusion-functions-nested = { workspace = true } insta = { workspace = true } diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 5a43d7118d63..b2fc2e494851 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -35,6 +35,7 @@ pub mod limit_pushdown; pub mod limited_distinct_aggregation; pub mod optimizer; pub mod output_requirements; +pub mod progressive_evaluation; pub mod projection_pushdown; pub mod pruning; pub mod sanity_checker; diff --git a/datafusion/physical-optimizer/src/progressive_evaluation.rs b/datafusion/physical-optimizer/src/progressive_evaluation.rs new file mode 100644 index 000000000000..60a130c30962 --- /dev/null +++ b/datafusion/physical-optimizer/src/progressive_evaluation.rs @@ -0,0 +1,109 @@ +// 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. + +//! TODO: physical optimizer run to conditionally swap the SPM with the ProgressiveEvalExec + +mod extract_ranges; +#[allow(dead_code)] +mod lexical_ranges; +mod statistics; +mod util; + +use itertools::Itertools; +use std::sync::Arc; + +use datafusion_common::{ + tree_node::{Transformed, TreeNode}, + Result, +}; +use datafusion_physical_plan::{ + sorts::sort_preserving_merge::SortPreservingMergeExec, union::UnionExec, + ExecutionPlan, +}; +use extract_ranges::extract_disjoint_ranges_from_plan; +use util::split_parquet_files; + +use crate::PhysicalOptimizerRule; + +#[allow(dead_code)] +#[derive(Debug)] +struct InsertProgressiveEval; + +impl PhysicalOptimizerRule for InsertProgressiveEval { + fn name(&self) -> &str { + "TBD" + } + + fn schema_check(&self) -> bool { + false + } + + fn optimize( + &self, + plan: Arc, + _config: &datafusion_common::config::ConfigOptions, + ) -> Result> { + plan.transform_up(|plan| { + // Find SortPreservingMergeExec + let Some(sort_preserving_merge_exec) = + plan.as_any().downcast_ref::() + else { + return Ok(Transformed::no(plan)); + }; + + // Split file groups to maximize potential disjoint ranges. + let new_inputs: Vec> = sort_preserving_merge_exec + .children() + .into_iter() + .map(|spm_child| { + Arc::clone(spm_child) + .transform_down(|plan| { + split_parquet_files(plan, sort_preserving_merge_exec.expr()) + }) + .map(|t| t.data) + }) + .try_collect()?; + let transformed_input_plan = Arc::new(UnionExec::new(new_inputs)) as _; + + // try to extract the lexical ranges for the input partitions + let Ok(Some(_lexical_ranges)) = extract_disjoint_ranges_from_plan( + sort_preserving_merge_exec.expr(), + &transformed_input_plan, + ) else { + return Ok(Transformed::no(plan)); + }; + + // confirm we still have the ordering needed for the SPM + assert!(transformed_input_plan + .properties() + .equivalence_properties() + .ordering_satisfy(sort_preserving_merge_exec.expr())); + + // Replace SortPreservingMergeExec with ProgressiveEvalExec + // TODO: have the ProgressiveEvalExec perform that partition mapping + // let progresive_eval_exec = Arc::new(ProgressiveEvalExec::new( + // transformed_input_plan, + // lexical_ranges, + // sort_preserving_merge_exec.fetch(), + // )); + // Ok(Transformed::yes(progresive_eval_exec)) + + Ok(Transformed::no(plan)) + }) + .map(|t| t.data) + } +} diff --git a/datafusion/physical-optimizer/src/progressive_evaluation/extract_ranges.rs b/datafusion/physical-optimizer/src/progressive_evaluation/extract_ranges.rs new file mode 100644 index 000000000000..3a37e7996bda --- /dev/null +++ b/datafusion/physical-optimizer/src/progressive_evaluation/extract_ranges.rs @@ -0,0 +1,684 @@ +// 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. + +//! Extract [`NonOverlappingOrderedLexicalRanges`] from different sources. + +use arrow::compute::SortOptions; +use arrow::datatypes::Schema; +use datafusion_common::{ + internal_datafusion_err, ColumnStatistics, Result, ScalarValue, Statistics, +}; +use datafusion_datasource::PartitionedFile; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::LexOrdering; +use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; +use std::sync::Arc; + +use super::lexical_ranges::{LexicalRange, NonOverlappingOrderedLexicalRanges}; +use super::statistics::{column_statistics_min_max, statistics_by_partition}; + +/// Attempt to extract LexicalRanges for the given sort keys and input plan +/// +/// Output will have N ranges where N is the number of output partitions +/// +/// Returns None if not possible to determine ranges +pub fn extract_disjoint_ranges_from_plan( + exprs: &LexOrdering, + input_plan: &Arc, +) -> Result> { + // if the ordering does not match, then we cannot confirm proper ranges + if !input_plan + .properties() + .equivalence_properties() + .ordering_satisfy(exprs) + { + return Ok(None); + } + + let num_input_partitions = partition_count(input_plan); + + // One builder for each output partition. + // Each builder will contain multiple sort keys + let mut builders = vec![LexicalRange::builder(); num_input_partitions]; + + // get partitioned stats for the plan + // TODO: this will be replace with the `ExecutionPlan::statistics_by_partition`, but it doesn't yet cover our + // use cases. (It does cover joins, which we don't address.) + let partitioned_stats = statistics_by_partition(input_plan.as_ref())?; + + // add per sort key + for sort_expr in exprs.iter() { + let Some(column) = sort_expr.expr.as_any().downcast_ref::() else { + return Ok(None); + }; + let Ok(col_idx) = input_plan.schema().index_of(column.name()) else { + return Ok(None); + }; + + // add per partition + for (builder, stats_for_partition) in builders.iter_mut().zip(&partitioned_stats) + { + let Some((min, max)) = min_max_for_stats_with_sort_options( + col_idx, + &sort_expr.options, + stats_for_partition, + )? + else { + return Ok(None); + }; + builder.push(min, max); + } + } + + let sort_options = exprs.iter().map(|e| e.options).collect::>(); + let ranges_per_partition = builders + .into_iter() + .map(|builder| builder.build()) + .collect::>(); + + NonOverlappingOrderedLexicalRanges::try_new(&sort_options, ranges_per_partition) +} + +fn partition_count(plan: &Arc) -> usize { + plan.output_partitioning().partition_count() +} + +// Returns the min and max value for the [`Statistics`], +/// and handles null values based on [`SortOptions`]. +/// +/// Returns None if statistics are absent/unknown. +fn min_max_for_stats_with_sort_options( + col_idx: usize, + sort_options: &SortOptions, + stats: &Statistics, +) -> Result> { + // Check if the column is a constant value according to the equivalence properties (TODO) + + let Some(ColumnStatistics { + max_value, + min_value, + null_count, + .. + }) = stats.column_statistics.get(col_idx) + else { + return Err(internal_datafusion_err!( + "extracted statistics is missing @{:?}, only has {:?} columns", + col_idx, + stats.column_statistics.len() + )); + }; + + let (Some(min), Some(max)) = (min_value.get_value(), max_value.get_value()) else { + return Ok(None); + }; + + let mut min = min.clone(); + let mut max = max.clone(); + if *null_count.get_value().unwrap_or(&0) > 0 { + let nulls_as_min = !sort_options.descending && sort_options.nulls_first // ASC nulls first + || sort_options.descending && !sort_options.nulls_first; // DESC nulls last + + // Get the typed null value for the data type of min/max + let null: ScalarValue = min.data_type().try_into()?; + + if nulls_as_min { + min = null; + } else { + max = null; + } + } + + Ok(Some((min, max))) +} + +/// Attempt to extract LexicalRanges for the given sort keys and partitioned files +/// +/// Output will have N ranges where N is the number of partitioned files +/// +/// Returns None if not possible to determine disjoint ranges +pub fn extract_ranges_from_files( + exprs: &LexOrdering, + files: &Vec<&PartitionedFile>, + schema: Arc, +) -> Result> { + let num_input_partitions = files.len(); + + // one builder for each output partition + let mut builders = vec![LexicalRange::builder(); num_input_partitions]; + for sort_expr in exprs.iter() { + let Some(column) = sort_expr.expr.as_any().downcast_ref::() else { + return Ok(None); + }; + let col_name = column.name(); + + for (file, builder) in files.iter().zip(builders.iter_mut()) { + let Some((min, max)) = min_max_for_partitioned_file(col_name, file, &schema)? + else { + return Ok(None); + }; + builder.push(min, max); + } + } + + let sort_options = exprs.iter().map(|e| e.options).collect::>(); + let ranges_per_partition = builders + .into_iter() + .map(|builder| builder.build()) + .collect::>(); + + NonOverlappingOrderedLexicalRanges::try_new(&sort_options, ranges_per_partition) +} + +/// Returns the min and max value for the specified partitioned file. +/// +/// Eventually this will not be required, since we will generalize the solution to not require DAG plan modification. +fn min_max_for_partitioned_file( + col_name: &str, + file: &PartitionedFile, + schema: &Arc, +) -> Result> { + let (Some((col_idx, _)), Some(file_stats)) = + (schema.fields().find(col_name), &file.statistics) + else { + return Ok(None); + }; + let col_stats = file_stats.column_statistics[col_idx].clone(); + Ok(column_statistics_min_max(col_stats)) +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::progressive_evaluation::util::test_utils::{ + parquet_exec_with_sort_with_statistics, + parquet_exec_with_sort_with_statistics_and_schema, sort_exec, union_exec, + SortKeyRange, + }; + use std::fmt::{Debug, Display, Formatter}; + + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field}; + use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; + use datafusion_physical_plan::displayable; + use datafusion_physical_plan::expressions::col; + + use insta::assert_snapshot; + + /// test with three partition ranges that are disjoint (non overlapping) + fn test_case_disjoint_3() -> TestCaseBuilder { + TestCaseBuilder::new() + .with_key_range(SortKeyRange { + min: Some(1000), + max: Some(2000), + null_count: 0, + }) + .with_key_range(SortKeyRange { + min: Some(2001), + max: Some(3000), + null_count: 0, + }) + .with_key_range(SortKeyRange { + min: Some(3001), + max: Some(3500), + null_count: 0, + }) + } + + /// test with three partition ranges that are NOT disjoint (are overlapping) + fn test_case_overlapping_3() -> TestCaseBuilder { + TestCaseBuilder::new() + .with_key_range(SortKeyRange { + min: Some(1000), + max: Some(2010), + null_count: 0, + }) + .with_key_range(SortKeyRange { + min: Some(2001), + max: Some(3000), + null_count: 0, + }) + .with_key_range(SortKeyRange { + min: Some(3001), + max: Some(3500), + null_count: 0, + }) + } + + #[test] + fn test_union_sort_union_disjoint_ranges_asc() { + assert_snapshot!( + test_case_disjoint_3() + .with_descending(false) + .with_sort_expr("a") + .union_sort_union_plan(), + @r" + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Input Ranges + (Some(1000))->(Some(2000)) + (Some(2001))->(Some(3000)) + (Some(3001))->(Some(3500)) + Output Ranges: [0, 1] + (1000)->(2000) + (2001)->(3500) + "); + } + + #[test] + fn test_union_sort_union_overlapping_ranges_asc() { + assert_snapshot!( + test_case_overlapping_3() + .with_descending(false) + .with_sort_expr("a") + .union_sort_union_plan(), + @r" + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Input Ranges + (Some(1000))->(Some(2010)) + (Some(2001))->(Some(3000)) + (Some(3001))->(Some(3500)) + No disjoint ranges found + "); + } + + #[test] + fn test_union_sort_union_disjoint_ranges_desc_nulls_first() { + assert_snapshot!( + test_case_disjoint_3() + .with_descending(true) + .with_sort_expr("a") + .union_sort_union_plan(), + @r" + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 DESC], file_type=parquet + SortExec: expr=[a@0 DESC], preserve_partitioning=[false] + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 DESC], file_type=parquet + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 DESC], file_type=parquet + + Input Ranges + (Some(1000))->(Some(2000)) + (Some(2001))->(Some(3000)) + (Some(3001))->(Some(3500)) + Output Ranges: [1, 0] + (2001)->(3500) + (1000)->(2000) + "); + } + + #[test] + fn test_union_sort_union_overlapping_ranges_desc_nulls_first() { + assert_snapshot!( + test_case_overlapping_3() + .with_descending(true) + .with_sort_expr("a") + .union_sort_union_plan(), + @r" + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 DESC], file_type=parquet + SortExec: expr=[a@0 DESC], preserve_partitioning=[false] + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 DESC], file_type=parquet + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 DESC], file_type=parquet + + Input Ranges + (Some(1000))->(Some(2010)) + (Some(2001))->(Some(3000)) + (Some(3001))->(Some(3500)) + No disjoint ranges found + "); + } + + // default is NULLS FIRST so try NULLS LAST + #[test] + fn test_union_sort_union_disjoint_ranges_asc_nulls_last() { + assert_snapshot!( + test_case_disjoint_3() + .with_descending(false) + .with_nulls_first(false) + .with_sort_expr("a") + .union_sort_union_plan(), + @r" + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=parquet + SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=parquet + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=parquet + + Input Ranges + (Some(1000))->(Some(2000)) + (Some(2001))->(Some(3000)) + (Some(3001))->(Some(3500)) + Output Ranges: [0, 1] + (1000)->(2000) + (2001)->(3500) + "); + } + + // default is NULLS FIRST so try NULLS LAST + #[test] + fn test_union_sort_union_overlapping_ranges_asc_nulls_last() { + assert_snapshot!( + test_case_overlapping_3() + .with_descending(false) + .with_nulls_first(false) + .with_sort_expr("a") + .union_sort_union_plan(), + @r" + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=parquet + SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=parquet + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=parquet + + Input Ranges + (Some(1000))->(Some(2010)) + (Some(2001))->(Some(3000)) + (Some(3001))->(Some(3500)) + No disjoint ranges found + "); + } + + #[test] + fn test_union_disjoint_ranges_asc() { + assert_snapshot!( + test_case_disjoint_3() + .with_descending(false) + .with_sort_expr("a") + .union_plan(), + @r" + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + DataSourceExec: file_groups={2 groups: [[0.parquet], [1.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Input Ranges + (Some(1000))->(Some(2000)) + (Some(2001))->(Some(3000)) + (Some(3001))->(Some(3500)) + Output Ranges: [0, 1, 2] + (1000)->(2000) + (2001)->(3000) + (3001)->(3500) + "); + } + + #[test] + fn test_union_overlapping_ranges_asc() { + assert_snapshot!( + test_case_overlapping_3() + .with_descending(false) + .with_sort_expr("a") + .union_plan(), + @r" + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + DataSourceExec: file_groups={2 groups: [[0.parquet], [1.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Input Ranges + (Some(1000))->(Some(2010)) + (Some(2001))->(Some(3000)) + (Some(3001))->(Some(3500)) + No disjoint ranges found + "); + } + + #[test] + fn test_union_disjoint_ranges_desc() { + assert_snapshot!( + test_case_disjoint_3() + .with_descending(true) + .with_sort_expr("a") + .union_plan(), + @r" + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 DESC], file_type=parquet + DataSourceExec: file_groups={2 groups: [[0.parquet], [1.parquet]]}, projection=[a], output_ordering=[a@0 DESC], file_type=parquet + + Input Ranges + (Some(1000))->(Some(2000)) + (Some(2001))->(Some(3000)) + (Some(3001))->(Some(3500)) + Output Ranges: [2, 1, 0] + (3001)->(3500) + (2001)->(3000) + (1000)->(2000) + "); + } + + #[test] + fn test_union_overlapping_ranges_desc() { + assert_snapshot!( + test_case_overlapping_3() + .with_descending(true) + .with_sort_expr("a") + .union_plan(), + @r" + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 DESC], file_type=parquet + DataSourceExec: file_groups={2 groups: [[0.parquet], [1.parquet]]}, projection=[a], output_ordering=[a@0 DESC], file_type=parquet + + Input Ranges + (Some(1000))->(Some(2010)) + (Some(2001))->(Some(3000)) + (Some(3001))->(Some(3500)) + No disjoint ranges found + "); + } + + /// Helper for building up patterns for testing statistics extraction from + /// ExecutionPlans + #[derive(Debug)] + struct TestCaseBuilder { + input_ranges: Vec, + sort_options: SortOptions, + sort_exprs: Vec, + schema: Arc, + } + + impl TestCaseBuilder { + /// Creates a new `TestCaseBuilder` instance with default values. + fn new() -> Self { + let schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, true)])); + + Self { + input_ranges: vec![], + sort_options: SortOptions::default(), + sort_exprs: vec![], + schema, + } + } + + /// Add a key range + pub fn with_key_range(mut self, key_range: SortKeyRange) -> Self { + self.input_ranges.push(key_range); + self + } + + /// set SortOptions::descending flag + pub fn with_descending(mut self, descending: bool) -> Self { + self.sort_options.descending = descending; + self + } + + /// set SortOptions::nulls_first flag + pub fn with_nulls_first(mut self, nulls_first: bool) -> Self { + self.sort_options.nulls_first = nulls_first; + self + } + + /// Add a sort expression to the ordering (created with the current SortOptions) + pub fn with_sort_expr(mut self, column_name: &str) -> Self { + let expr = PhysicalSortExpr::new( + col(column_name, &self.schema).unwrap(), + self.sort_options, + ); + self.sort_exprs.push(expr); + self + } + + /// Build a test physical plan like the following, and extract disjoint ranges from it: + /// + /// ```text + /// UNION + /// ParquetExec (key_ranges[0]) (range_a) + /// SORT + /// UNION + /// ParquetExec (key_ranges[1]) (range_b_1) + /// ParquetExec (key_ranges[2]) (range_b_2) + /// ``` + fn union_sort_union_plan(self) -> TestResult { + let Self { + input_ranges, + sort_options: _, // used to create sort exprs, nothere + sort_exprs, + schema, + } = self; + let lex_ordering = LexOrdering::new(sort_exprs); + + assert_eq!(input_ranges.len(), 3); + let range_a = &input_ranges[0]; + let range_b_1 = &input_ranges[1]; + let range_b_2 = &input_ranges[2]; + + let datasrc_a = parquet_exec_with_sort_with_statistics_and_schema( + &schema, + vec![lex_ordering.clone()], + &[range_a], + ); + + let datasrc_b1 = parquet_exec_with_sort_with_statistics_and_schema( + &schema, + vec![lex_ordering.clone()], + &[range_b_1], + ); + let datasrc_b2 = parquet_exec_with_sort_with_statistics_and_schema( + &schema, + vec![lex_ordering.clone()], + &[range_b_2], + ); + let b = sort_exec( + &lex_ordering, + &union_exec(vec![datasrc_b1, datasrc_b2]), + false, + ); + + let plan = union_exec(vec![datasrc_a, b]); + + let actual = extract_disjoint_ranges_from_plan(&lex_ordering, &plan) + .expect("Error extracting disjoint ranges from plan"); + TestResult { + input_ranges, + plan, + actual, + } + } + + /// Build a test physical plan like the following, and extract disjoint ranges from it: + /// + /// ```text + /// UNION + /// ParquetExec (key_ranges[0]) (range_a) + /// ParquetExec (key_ranges[1], key_ranges[2]) (range_b_1, range_b_2) + /// ``` + fn union_plan(self) -> TestResult { + let Self { + input_ranges, + sort_options: _, // used to create sort exprs, nothere + sort_exprs, + schema, + } = self; + + let lex_ordering = LexOrdering::new(sort_exprs); + + assert_eq!(input_ranges.len(), 3); + let range_a = &input_ranges[0]; + let range_b_1 = &input_ranges[1]; + let range_b_2 = &input_ranges[2]; + let datasrc_a = parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + &[range_a], + ); + let datasrc_b = parquet_exec_with_sort_with_statistics_and_schema( + &schema, + vec![lex_ordering.clone()], + &[range_b_1, range_b_2], + ); + + let plan = union_exec(vec![datasrc_a, datasrc_b]); + + let actual = extract_disjoint_ranges_from_plan(&lex_ordering, &plan) + .expect("Error extracting disjoint ranges from plan"); + TestResult { + input_ranges, + plan, + actual, + } + } + } + + /// Result of running a test case, including the input ranges, the execution + /// plan, and the actual disjoint ranges found. + /// + /// This struct implements `Display` to provide a formatted output of the + /// test case results that can be easily compared using `insta` snapshots. + struct TestResult { + input_ranges: Vec, + plan: Arc, + actual: Option, + } + + impl TestResult {} + + impl Display for TestResult { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + let displayable_plan = displayable(self.plan.as_ref()).indent(false); + writeln!(f, "{}", displayable_plan)?; + + writeln!(f, "Input Ranges")?; + for range in &self.input_ranges { + writeln!(f, " {}", range)?; + } + + match self.actual.as_ref() { + Some(actual) => { + writeln!(f, "Output Ranges: {:?}", actual.indices())?; + for range in actual.ordered_ranges() { + writeln!(f, " {}", range)?; + } + } + None => { + writeln!(f, "No disjoint ranges found")?; + } + } + Ok(()) + } + } +} diff --git a/datafusion/physical-optimizer/src/progressive_evaluation/lexical_ranges.rs b/datafusion/physical-optimizer/src/progressive_evaluation/lexical_ranges.rs new file mode 100644 index 000000000000..e9c978c96b12 --- /dev/null +++ b/datafusion/physical-optimizer/src/progressive_evaluation/lexical_ranges.rs @@ -0,0 +1,1017 @@ +// 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. + +//! [`NonOverlappingOrderedLexicalRanges`] represents ranges of lexically ordered values. + +use arrow::array::ArrayRef; +use arrow::compute::SortOptions; +use arrow::row::{Row, RowConverter, Rows, SortField}; +use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; +use std::fmt::Display; +use std::sync::Arc; + +/// Represents a range of sort key values within a lexical space. +/// +/// # Lexical Space +/// +/// The "Lexical Space" is all possible values of columns in a sort order (set +/// of sort expressions). +/// +/// For example, given data with a sort order of `A ASC, B ASC` +/// (`A` ascending, `B` ascending), then the lexical space is all the unique +/// combinations of `(A, B)`. +/// +/// # Lexical Range +/// +/// The "lexical range" is defined by two points in a lexical space (the +/// minimum and maximum sort key values for some range) +/// +/// For example, for data like +/// +/// |`a`| `b` | +/// |---|-----| +/// | 1 | 100 | +/// | 1 | 200 | +/// | 1 | 300 | +/// | 2 | 100 | +/// | 2 | 200 | +/// | 3 | 50 | +/// +/// The lexical range is +/// * `min --> max` +/// * `(a_min, b_min) -> (a_max, b_max)` +/// * `(1,100) --> (3,50)` +#[derive(Debug, Default, Clone)] +pub struct LexicalRange { + /// The minimum multi-column value in the lexical space (one `ScalarValue` + /// for each sort key) + min: Vec, + /// The maximum multi-column value in the lexical space (one `ScalarValue` + /// for each sort key) + max: Vec, +} + +impl LexicalRange { + /// Create a [`LexicalRangeBuilder`] + pub fn builder() -> LexicalRangeBuilder { + LexicalRangeBuilder::new() + } + + /// Create a new [`LexicalRange`] with the same min and max values. + pub fn new_from_constants(constants: Vec) -> Self { + Self { + min: constants.clone(), + max: constants, + } + } +} + +impl Display for LexicalRange { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + "({})->({})", + self.min + .iter() + .map(|v| v.to_string()) + .collect::>() + .join(","), + self.max + .iter() + .map(|v| v.to_string()) + .collect::>() + .join(",") + ) + } +} + +/// Builder for [`LexicalRange`] +/// +/// This builds up a multi-column min/max range one pair at a time. +#[derive(Debug, Default, Clone)] +pub struct LexicalRangeBuilder { + inner: LexicalRange, +} + +impl LexicalRangeBuilder { + pub fn new() -> Self { + Self::default() + } + + pub fn build(self) -> LexicalRange { + self.inner + } + + /// Adds a min and max to the end of the in-progress ranges + pub fn push(&mut self, min: ScalarValue, max: ScalarValue) { + self.inner.min.push(min); + self.inner.max.push(max); + } +} + +/// Represents a set of non overlapping [`LexicalRange`]s ordered according to +/// minimum value. +/// +/// # Single Column +/// +/// For example given a set of ranges +/// ```text +/// (11, 20) // partition 0 +/// (1, 10) // partition 1 +/// (21, 30) // partition 2 +/// ``` +/// +/// Then `[1, 0, 2]` is a non-overlapping ordered set of lexical ranges. When +/// ordered by minimum value: `(1, 10)` comes first, then `(11, 20)` and then +/// `(21, 30)`. +/// +/// There are no `NonOverlappingOrderedLexicalRanges` if the ranges are not +/// disjoint (they overlap). For example, the following ranges overlap between +/// 11 and 15: +/// +/// ```text +/// (11, 20) // partition 0 +/// (1, 15) // partition 1 +/// ``` +/// +#[derive(Debug)] +pub struct NonOverlappingOrderedLexicalRanges { + /// lexical ranges. + /// + /// These are typically used to represent the value ranges of each + /// DataFusion (not IOx) partition + value_ranges: Vec, + + /// Indexes into `value_ranges` that define a non overlapping ordering by + /// minimum value. + indices: Vec, +} + +impl Display for NonOverlappingOrderedLexicalRanges { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "LexicalRanges\n {:?}", self.indices,) + } +} + +impl NonOverlappingOrderedLexicalRanges { + /// Attempt to create a new [`NonOverlappingOrderedLexicalRanges`] given the + /// specified [`SortOptions`]. + /// + /// See struct documentation for more details + /// + /// Returns None if: + /// * - There are no ranges + /// * - The ranges are overlap in lexical space (are not disjoint) + /// + /// Returns Err if there is an error converting values. + pub fn try_new( + sort_options: &[SortOptions], + value_ranges: Vec, + ) -> Result> { + if value_ranges.is_empty() { + return Ok(None); + } + + // convert to rows, for row ordering per sort key + let rows = ConvertedRows::try_new_from_lexical_ranges( + sort_options, + value_ranges.clone(), + )?; + + // order by minimum value + let mut indices = (0..rows.num_rows()).collect::>(); + indices.sort_by_key(|&i| rows.min_row(i)); + + // check that the ranges are disjoint + if !rows.are_disjoint(&indices) { + return Ok(None); + }; + + Ok(Some(Self { + value_ranges, + indices, + })) + } + + /// Indices that define an ordered list of non overlapping value ranges. + pub fn indices(&self) -> &[usize] { + &self.indices + } + + /// Iterator over the in lexical ranges in order + pub fn ordered_ranges(&self) -> impl Iterator { + self.indices.iter().map(|i| &self.value_ranges[*i]) + } +} + +/// Result of converting multiple-column ScalarValue rows to columns. +#[derive(Debug)] +struct ConvertedRows { + /// converter for mins and maxes + /// + /// Must use the same conveted rows for both, otherwise they cannot be + /// compared. + converter: RowConverter, + + mins: Rows, + maxes: Rows, +} + +impl ConvertedRows { + /// Create new [`ConvertedRows`] from the vector of sort keys and specified options. + /// + /// Keys are in the format `VecPerPartition>` + fn try_new( + sort_options: &[SortOptions], + min_keys: Vec>, + max_keys: Vec>, + ) -> Result { + if sort_options.len() != min_keys[0].len() { + return internal_err!( + "Expected number of sort options ({}) to match number of sort keys in min_keys ({})", + sort_options.len(), + min_keys[0].len() + ); + } + if sort_options.len() != max_keys[0].len() { + return internal_err!( + "Expected number of sort options ({}) to match number of sort keys in max_keys ({})", + sort_options.len(), + max_keys.len() + ); + } + + // build converter using min keys + let arrays = pivot_to_arrays(min_keys)?; + let converter_fields = arrays + .iter() + .zip(sort_options.iter()) + .map(|(a, options)| { + SortField::new_with_options(a.data_type().clone(), *options) + }) + .collect::>(); + let converter = RowConverter::new(converter_fields)?; + let mins = converter.convert_columns(&arrays)?; + + // build maxes + let arrays = pivot_to_arrays(max_keys)?; + let maxes = converter.convert_columns(&arrays)?; + + Ok(Self { + converter, + mins, + maxes, + }) + } + + fn try_new_from_lexical_ranges( + sort_options: &[SortOptions], + lexical_ranges: Vec, + ) -> Result { + // convert to min/maxes, as VecPerPartition + let (mins, maxes) = lexical_ranges.clone().into_iter().fold( + (vec![], vec![]), + |(mut mins, mut maxes), partition_range| { + let LexicalRange { + min: min_per_sort_key, + max: max_per_sort_key, + } = partition_range; + mins.push(min_per_sort_key); + maxes.push(max_per_sort_key); + (mins, maxes) + }, + ); + Self::try_new(sort_options, mins, maxes) + } + + /// Return the number of partitions (rows) in the converted rows. + fn num_rows(&self) -> usize { + self.mins.num_rows() + } + + /// Return the min (as [`Row`]) at the specified index + fn min_row(&self, index: usize) -> Row<'_> { + self.mins.row(index) + } + + /// Return the max (as [`Row`]) at the specified index + fn max_row(&self, index: usize) -> Row<'_> { + self.maxes.row(index) + } + + /// Return the min value at the specified index as a list of single + /// row arrays + /// + /// Used for debugging + fn min_value(&self, index: usize) -> Result> { + let values = self + .converter + .convert_rows([self.min_row(index)]) + .map_err(|e| DataFusionError::ArrowError(e, None))?; + Ok(values.iter().map(Arc::clone).collect::>()) + } + + /// Return the max value at the specified index as a list of single + /// row arrays + /// + /// Used for debugging + fn max_value(&self, index: usize) -> Result> { + let values = self + .converter + .convert_rows([self.max_row(index)]) + .map_err(|e| DataFusionError::ArrowError(e, None))?; + Ok(values.iter().map(Arc::clone).collect::>()) + } + + /// Return true if ranges are disjoint assuming the ranges are sorted by + /// `ordered_by_min_partition_indices` + /// + /// This checks each pair of adjacent ranges in the sorted order. + /// + /// (0 -> 10) (11 -> 20) (21 -> 30) => disjoint=true + /// output = true + /// + /// (0 -> 10) (10 -> 20) (21 -> 30) => disjoint=true because 10==10 and does not invalidate the ordering + /// output = true + /// + /// (0 -> 13) (12 -> 20) (21 -> 30) => disjoint=false + /// output = None + /// + fn are_disjoint(&self, ordered_by_min_partition_indices: &[usize]) -> bool { + for index_index in 1..ordered_by_min_partition_indices.len() { + let index = ordered_by_min_partition_indices[index_index]; + let prev_index = ordered_by_min_partition_indices[index_index - 1]; + + // Ordering is by sort key, and may be desc. + // Therefore need to confirm that the min & max of the current range is greater than the previous range. + let start_exclusive = self.min_row(index) > self.min_row(prev_index) + && self.min_row(index) >= self.max_row(prev_index); + let end_exclusive = self.max_row(index) >= self.min_row(prev_index) + && self.max_row(index) > self.max_row(prev_index); + + if !(start_exclusive && end_exclusive) { + return false; + } + } + true + } +} + +/// Convert a multi-column ScalarValue row to columns +fn pivot_to_arrays(keys: Vec>) -> Result> { + let mut arrays = vec![]; + for col in 0..keys[0].len() { + let mut column = vec![]; + for row in &keys { + // todo avoid this clone (with take) + column.push(row[col].clone()); + } + arrays.push(ScalarValue::iter_to_array(column)?) + } + Ok(arrays) +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::compute::SortOptions; + use itertools::Itertools; + + struct TestCase { + partitions: Vec, + num_sort_keys: usize, + name: &'static str, + expected_ranges_per_partition: Vec<&'static str>, // before ordering + expect_disjoint: bool, + expected_ordered_indices: Vec, // after ordering + } + + impl TestCase { + fn run(self) { + println!("Beginning test : {}", self.name); + // Test: confirm found proper lexical ranges + let lexical_ranges = self.build_lexical_ranges(); + let expected = self + .expected_ranges_per_partition + .iter() + .map(|str| str.to_string()) + .collect_vec(); + let actual = lexical_ranges + .iter() + .map(|range| format!("{range}")) + .collect_vec(); + assert_eq!( + actual, expected, + "ERROR {}: expected ranges {:?} but found {:?}", + self.name, expected, actual + ); + + // Test: confirm found proper non overlapping (or not) ranges per given sort ordering + [ + TestSortOption::AscNullsFirst, + TestSortOption::AscNullsLast, + TestSortOption::DescNullsFirst, + TestSortOption::DescNullsLast, + ].into_iter().for_each(|sort_ordering| { + if let Some(nonoverlapping) = NonOverlappingOrderedLexicalRanges::try_new(&sort_ordering.sort_options(self.num_sort_keys), lexical_ranges.clone()).expect("should not error") { + assert!(self.expect_disjoint, + "ERROR {} for {:?}: expected ranges to overlap, instead found disjoint ranges", + self.name, &sort_ordering); + + let expected_ordered_indices = self.find_expected_indices(&sort_ordering); + assert_eq!(expected_ordered_indices, nonoverlapping.indices(), + "ERROR {} for {:?}: expected to find indices ordered {:?}, instead found ordering {:?}", + self.name, &sort_ordering, expected_ordered_indices, nonoverlapping.indices()); + } else { + assert!(!self.expect_disjoint, + "ERROR {} for {:?}: expected to find disjoint ranges, instead could either not detect ranges or found overlapping ranges", + self.name, &sort_ordering); + }; + }); + } + + fn build_lexical_ranges(&self) -> Vec { + self.partitions + .iter() + .map(|partition| { + let mut builder = LexicalRange::builder(); + for SortKeyRange { min, max } in &partition.range_per_sort_key { + builder.push(min.clone(), max.clone()); + } + builder.build() + }) + .collect_vec() + } + + fn find_expected_indices(&self, sort_ordering: &TestSortOption) -> &[usize] { + self.expected_ordered_indices + .iter() + .find(|ord| ord.sort_ordering == *sort_ordering) + .expect("should have expected outcome") + .expected_indices + .as_ref() + } + } + + struct TestPartition { + range_per_sort_key: Vec, + } + + /// Range of a sort key. Note that this is not impacted by directionality of ordering (e.g. [`SortOptions`]). + struct SortKeyRange { + min: ScalarValue, + max: ScalarValue, + } + + fn build_partition_with_single_sort_key( + ints: (Option, Option), + ) -> TestPartition { + let range_per_sort_key = vec![SortKeyRange { + min: ScalarValue::Int64(ints.0), + max: ScalarValue::Int64(ints.1), + }]; + TestPartition { range_per_sort_key } + } + + /// Build min/maxes for a partition with three sort keys: + /// 1. An integer sort key + /// 2. A string sort key + /// 3. A timestamp sort key (nanoseconds) + fn build_partition_with_multiple_sort_keys( + ints: (Option, Option), + strings: (Option<&str>, Option<&str>), + times: (Option, Option), + ) -> TestPartition { + let range_per_sort_key = vec![ + SortKeyRange { + min: ScalarValue::Int64(ints.0), + max: ScalarValue::Int64(ints.1), + }, + SortKeyRange { + min: ScalarValue::from(strings.0), + max: ScalarValue::from(strings.1), + }, + SortKeyRange { + min: ScalarValue::TimestampNanosecond(times.0, None), + max: ScalarValue::TimestampNanosecond(times.1, None), + }, + ]; + TestPartition { range_per_sort_key } + } + + #[derive(Debug, PartialEq)] + enum TestSortOption { + AscNullsLast, + AscNullsFirst, + DescNullsLast, + DescNullsFirst, + } + + impl TestSortOption { + fn sort_options(&self, len: usize) -> Vec { + match self { + Self::AscNullsLast => std::iter::repeat_n( + SortOptions { + descending: false, + nulls_first: false, + }, + len, + ) + .collect_vec(), + Self::AscNullsFirst => std::iter::repeat_n( + SortOptions { + descending: false, + nulls_first: true, + }, + len, + ) + .collect_vec(), + Self::DescNullsLast => std::iter::repeat_n( + SortOptions { + descending: true, + nulls_first: false, + }, + len, + ) + .collect_vec(), + Self::DescNullsFirst => std::iter::repeat_n( + SortOptions { + descending: true, + nulls_first: true, + }, + len, + ) + .collect_vec(), + } + } + } + + struct ExpectedOrderedIndices { + /// the ordering (e.g. [`SortOptions`]) applied to all columns in the sort key. + sort_ordering: TestSortOption, + /// Expected outcome ordering with this sort_ordering applied. + expected_indices: Vec, + } + + impl From<(TestSortOption, Vec)> for ExpectedOrderedIndices { + fn from(value: (TestSortOption, Vec)) -> Self { + Self { + sort_ordering: value.0, + expected_indices: value.1, + } + } + } + + #[test] + fn test_disjointness_single_key() { + let cases = [ + TestCase { + partitions: vec![ + build_partition_with_single_sort_key((Some(1), Some(10))), + build_partition_with_single_sort_key((Some(2), Some(10))), + build_partition_with_single_sort_key((Some(0), Some(0))), + ], + num_sort_keys: 1, + name: "order_by_single_sort_key__overlapping", + expected_ranges_per_partition: vec!["(1)->(10)", "(2)->(10)", "(0)->(0)"], + expect_disjoint: false, + expected_ordered_indices: vec![], + }, + TestCase { + partitions: vec![ + build_partition_with_single_sort_key((Some(1), Some(10))), + build_partition_with_single_sort_key((Some(11), Some(20))), + build_partition_with_single_sort_key((Some(0), Some(0))), + ], + num_sort_keys: 1, + name: "order_by_single_sort_key__disjoint", + expected_ranges_per_partition: vec![ + "(1)->(10)", + "(11)->(20)", + "(0)->(0)", + ], + expect_disjoint: true, + expected_ordered_indices: vec![ + (TestSortOption::AscNullsFirst, vec![2, 0, 1]).into(), + (TestSortOption::AscNullsLast, vec![2, 0, 1]).into(), + (TestSortOption::DescNullsFirst, vec![1, 0, 2]).into(), + (TestSortOption::DescNullsLast, vec![1, 0, 2]).into(), + ], + }, + ]; + + cases.into_iter().for_each(|test_case| test_case.run()); + } + + #[test] + fn test_disjointness_multiple_sort_keys() { + let cases = [ + /* Using the first sort key, an integer, as the decider. */ + TestCase { + partitions: vec![ + build_partition_with_multiple_sort_keys( + (Some(1), Some(10)), + (Some("same"), Some("same")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(2), Some(10)), + (Some("same"), Some("same")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(0), Some(0)), + (Some("same"), Some("same")), + (Some(1), Some(1)), + ), + ], + num_sort_keys: 3, + name: "order_by_first_sort_key__overlapping", + expected_ranges_per_partition: vec![ + "(1,same,1)->(10,same,1)", + "(2,same,1)->(10,same,1)", + "(0,same,1)->(0,same,1)", + ], + expect_disjoint: false, + expected_ordered_indices: vec![], + }, + TestCase { + partitions: vec![ + build_partition_with_multiple_sort_keys( + (Some(1), Some(10)), + (Some("same"), Some("same")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(11), Some(20)), + (Some("same"), Some("same")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(0), Some(0)), + (Some("same"), Some("same")), + (Some(1), Some(1)), + ), + ], + num_sort_keys: 3, + name: "order_by_first_sort_key__disjoint", + expected_ranges_per_partition: vec![ + "(1,same,1)->(10,same,1)", + "(11,same,1)->(20,same,1)", + "(0,same,1)->(0,same,1)", + ], + expect_disjoint: true, + expected_ordered_indices: vec![ + (TestSortOption::AscNullsFirst, vec![2, 0, 1]).into(), + (TestSortOption::AscNullsLast, vec![2, 0, 1]).into(), + (TestSortOption::DescNullsFirst, vec![1, 0, 2]).into(), + (TestSortOption::DescNullsLast, vec![1, 0, 2]).into(), + ], + }, + /* Using the middle sort key, a string, as the decider. */ + TestCase { + partitions: vec![ + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("a"), Some("d")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("f"), Some("g")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("c"), Some("e")), + (Some(1), Some(1)), + ), + ], + num_sort_keys: 3, + name: "order_by_middle_sort_key__overlapping", + expected_ranges_per_partition: vec![ + "(1,a,1)->(1,d,1)", + "(1,f,1)->(1,g,1)", + "(1,c,1)->(1,e,1)", + ], + expect_disjoint: false, + expected_ordered_indices: vec![], + }, + TestCase { + partitions: vec![ + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("a"), Some("b")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("f"), Some("g")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("c"), Some("e")), + (Some(1), Some(1)), + ), + ], + num_sort_keys: 3, + name: "order_by_middle_sort_key__disjoint", + expected_ranges_per_partition: vec![ + "(1,a,1)->(1,b,1)", + "(1,f,1)->(1,g,1)", + "(1,c,1)->(1,e,1)", + ], + expect_disjoint: true, + expected_ordered_indices: vec![ + (TestSortOption::AscNullsFirst, vec![0, 2, 1]).into(), + (TestSortOption::AscNullsLast, vec![0, 2, 1]).into(), + (TestSortOption::DescNullsFirst, vec![1, 2, 0]).into(), + (TestSortOption::DescNullsLast, vec![1, 2, 0]).into(), + ], + }, + /* Using the last sort key, a nanosecond timestamp, as the decider. */ + TestCase { + partitions: vec![ + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("same"), Some("same")), + (Some(50000000), Some(50000001)), + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("same"), Some("same")), + (Some(700000), Some(50000001)), + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("same"), Some("same")), + (Some(100000000), Some(100000001)), + ), + ], + num_sort_keys: 3, + name: "order_by_last_sort_key__overlapping", + expected_ranges_per_partition: vec![ + "(1,same,50000000)->(1,same,50000001)", + "(1,same,700000)->(1,same,50000001)", + "(1,same,100000000)->(1,same,100000001)", + ], + expect_disjoint: false, + expected_ordered_indices: vec![], + }, + TestCase { + partitions: vec![ + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("same"), Some("same")), + (Some(50000000), Some(50000001)), + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("same"), Some("same")), + (Some(700000), Some(7000001)), + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("same"), Some("same")), + (Some(100000000), Some(100000001)), + ), + ], + num_sort_keys: 3, + name: "order_by_last_sort_key__disjoint", + expected_ranges_per_partition: vec![ + "(1,same,50000000)->(1,same,50000001)", + "(1,same,700000)->(1,same,7000001)", + "(1,same,100000000)->(1,same,100000001)", + ], + expect_disjoint: true, + expected_ordered_indices: vec![ + (TestSortOption::AscNullsFirst, vec![1, 0, 2]).into(), + (TestSortOption::AscNullsLast, vec![1, 0, 2]).into(), + (TestSortOption::DescNullsFirst, vec![2, 0, 1]).into(), + (TestSortOption::DescNullsLast, vec![2, 0, 1]).into(), + ], + }, + ]; + + cases.into_iter().for_each(|test_case| test_case.run()); + } + + #[test] + fn test_disjointness_with_nulls() { + let cases = [ + TestCase { + partitions: vec![ + build_partition_with_single_sort_key((Some(1), Some(10))), + build_partition_with_single_sort_key((Some(2), Some(10))), + build_partition_with_single_sort_key((None, None)), + ], + num_sort_keys: 1, + name: "order_by_nulls__overlapping", + expected_ranges_per_partition: vec![ + "(1)->(10)", + "(2)->(10)", + "(NULL)->(NULL)", + ], + expect_disjoint: false, + expected_ordered_indices: vec![], + }, + TestCase { + partitions: vec![ + build_partition_with_single_sort_key((Some(1), Some(10))), + build_partition_with_single_sort_key((Some(11), Some(20))), + build_partition_with_single_sort_key((None, None)), + ], + num_sort_keys: 1, + name: "order_by_nulls__disjoint", + expected_ranges_per_partition: vec![ + "(1)->(10)", + "(11)->(20)", + "(NULL)->(NULL)", + ], + expect_disjoint: true, + expected_ordered_indices: vec![ + (TestSortOption::AscNullsFirst, vec![2, 0, 1]).into(), + (TestSortOption::AscNullsLast, vec![0, 1, 2]).into(), + (TestSortOption::DescNullsFirst, vec![2, 1, 0]).into(), + (TestSortOption::DescNullsLast, vec![1, 0, 2]).into(), + ], + }, + TestCase { + partitions: vec![ + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (None, Some("e")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("e"), None), + (Some(1), Some(1)), + ), + ], + num_sort_keys: 3, + name: "order_by_nulls__overlap", + expected_ranges_per_partition: vec![ + "(1,NULL,1)->(1,e,1)", + "(1,e,1)->(1,NULL,1)", + ], + expect_disjoint: false, + expected_ordered_indices: vec![], + }, + TestCase { + partitions: vec![ + build_partition_with_multiple_sort_keys( + (Some(10), Some(20)), + (None, Some("c")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(21), Some(22)), + (Some("e"), Some("f")), + (Some(2), Some(3)), + ), + ], + num_sort_keys: 3, + name: "order_by_nulls__disjoint2", + expected_ranges_per_partition: vec![ + "(10,NULL,1)->(20,c,1)", + "(21,e,2)->(22,f,3)", + ], + expect_disjoint: true, + expected_ordered_indices: vec![ + (TestSortOption::AscNullsFirst, vec![0, 1]).into(), + (TestSortOption::AscNullsLast, vec![0, 1]).into(), + (TestSortOption::DescNullsFirst, vec![1, 0]).into(), + (TestSortOption::DescNullsLast, vec![1, 0]).into(), + ], + }, + ]; + + cases.into_iter().for_each(|test_case| test_case.run()); + } + + #[test] + fn test_disjointness_with_touching_disjoint_ranges() { + let cases = [ + /* Using the first sort key, an integer, and has a start(current)==end(previous). */ + TestCase { + partitions: vec![ + build_partition_with_multiple_sort_keys( + (Some(1), Some(10)), // ends with 10 + (Some("same"), Some("same")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(10), Some(20)), // starts with 10 + (Some("same"), Some("same")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(0), Some(0)), + (Some("same"), Some("same")), + (Some(1), Some(1)), + ), + ], + num_sort_keys: 3, + name: "order_by_first_sort_key__disjoint_touching", + expected_ranges_per_partition: vec![ + "(1,same,1)->(10,same,1)", + "(10,same,1)->(20,same,1)", + "(0,same,1)->(0,same,1)", + ], + expect_disjoint: true, + expected_ordered_indices: vec![ + (TestSortOption::AscNullsFirst, vec![2, 0, 1]).into(), + (TestSortOption::AscNullsLast, vec![2, 0, 1]).into(), + (TestSortOption::DescNullsFirst, vec![1, 0, 2]).into(), + (TestSortOption::DescNullsLast, vec![1, 0, 2]).into(), + ], + }, + /* Using the middle sort key, a string, as the decider, and has a start(current)==end(previous) */ + TestCase { + partitions: vec![ + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("a"), Some("c")), // ends with c + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("f"), Some("g")), + (Some(1), Some(1)), + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("c"), Some("e")), // starts with c + (Some(1), Some(1)), + ), + ], + num_sort_keys: 3, + name: "order_by_middle_sort_key__disjoint_touching", + expected_ranges_per_partition: vec![ + "(1,a,1)->(1,c,1)", + "(1,f,1)->(1,g,1)", + "(1,c,1)->(1,e,1)", + ], + expect_disjoint: true, + expected_ordered_indices: vec![ + (TestSortOption::AscNullsFirst, vec![0, 2, 1]).into(), + (TestSortOption::AscNullsLast, vec![0, 2, 1]).into(), + (TestSortOption::DescNullsFirst, vec![1, 2, 0]).into(), + (TestSortOption::DescNullsLast, vec![1, 2, 0]).into(), + ], + }, + /* Using the last sort key, a nanosecond timestamp, as the decider, and has a start(current)==end(previous) */ + TestCase { + partitions: vec![ + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("same"), Some("same")), + (Some(50000000), Some(50000001)), // ends with 50000001 + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("same"), Some("same")), + (Some(700000), Some(7000001)), + ), + build_partition_with_multiple_sort_keys( + (Some(1), Some(1)), + (Some("same"), Some("same")), + (Some(50000001), Some(100000001)), // starts with 50000001 + ), + ], + num_sort_keys: 3, + name: "order_by_last_sort_key__disjoint_touching", + expected_ranges_per_partition: vec![ + "(1,same,50000000)->(1,same,50000001)", + "(1,same,700000)->(1,same,7000001)", + "(1,same,50000001)->(1,same,100000001)", + ], + expect_disjoint: true, + expected_ordered_indices: vec![ + (TestSortOption::AscNullsFirst, vec![1, 0, 2]).into(), + (TestSortOption::AscNullsLast, vec![1, 0, 2]).into(), + (TestSortOption::DescNullsFirst, vec![2, 0, 1]).into(), + (TestSortOption::DescNullsLast, vec![2, 0, 1]).into(), + ], + }, + ]; + + cases.into_iter().for_each(|test_case| test_case.run()); + } +} diff --git a/datafusion/physical-optimizer/src/progressive_evaluation/statistics.rs b/datafusion/physical-optimizer/src/progressive_evaluation/statistics.rs new file mode 100644 index 000000000000..8fe706284222 --- /dev/null +++ b/datafusion/physical-optimizer/src/progressive_evaluation/statistics.rs @@ -0,0 +1,1708 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use datafusion_common::{ + stats::Precision, ColumnStatistics, DataFusionError, Result, ScalarValue, Statistics, +}; +use datafusion_datasource::{file_scan_config::FileScanConfig, source::DataSourceExec}; +use datafusion_physical_plan::{ + coalesce_batches::CoalesceBatchesExec, + coalesce_partitions::CoalescePartitionsExec, + empty::EmptyExec, + filter::FilterExec, + limit::{GlobalLimitExec, LocalLimitExec}, + placeholder_row::PlaceholderRowExec, + projection::ProjectionExec, + repartition::RepartitionExec, + sorts::{sort::SortExec, sort_preserving_merge::SortPreservingMergeExec}, + union::UnionExec, + ExecutionPlan, PhysicalExpr, +}; +use fetch::apply_fetch; +use filter::apply_filter; +use itertools::Itertools; +use project_schema::{proj_exec_stats, project_select_subset_of_column_statistics}; +use util::{make_column_statistics_inexact, merge_stats_collection, partition_count}; + +mod fetch; +mod filter; +mod project_schema; +pub(crate) use project_schema::project_schema_onto_datasrc_statistics; +mod util; + +/// Return min max of a ColumnStatistics with precise values +pub fn column_statistics_min_max( + column_statistics: ColumnStatistics, +) -> Option<(ScalarValue, ScalarValue)> { + match (column_statistics.min_value, column_statistics.max_value) { + (Precision::Exact(min), Precision::Exact(max)) => Some((min, max)), + // the statistics values are absent or imprecise + _ => None, + } +} + +/// This matches the planned API for DataFusion's `PartitionedStatistics`. +pub type PartitionedStatistics = Vec>; + +pub(crate) trait PartitionStatistics { + /// Returns a [`Statistics`] per partition. + /// + /// This function matches the planned API for DataFusion's `ExecutionPlan::statistics_by_partition`. + fn statistics_by_partition(&self) -> Result; +} + +impl PartitionStatistics for EmptyExec { + fn statistics_by_partition(&self) -> Result { + let mut stats = Statistics::new_unknown(&self.schema()); + stats.num_rows = Precision::Exact(0); // tis empty + let data = Arc::new(stats); + + Ok(vec![data; partition_count(self)]) + } +} + +impl PartitionStatistics for PlaceholderRowExec { + fn statistics_by_partition(&self) -> Result { + // all partitions have the same single row, and same stats. + // refer to `PlaceholderRowExec::execute`. + #[expect(deprecated)] + let data = self.statistics()?.into(); + Ok(vec![data; partition_count(self)]) + } +} + +impl PartitionStatistics for DataSourceExec { + fn statistics_by_partition(&self) -> Result { + // Extract partitioned files from the Parquet, which is the datasource we use. + let Some(base_config) = + self.data_source().as_any().downcast_ref::() + else { + return Ok(unknown_statistics_by_partition(self)); + }; + + let file_schema = &base_config.file_schema; + let target_schema = Arc::clone(&self.schema()); + + // get per partition (a.k.a. file group) + let per_partition = base_config + .file_groups + .iter() + .map(|file_group| { + file_group + .iter() + .map(|file| { + // per file, get projected statistics + let statistics = if let Some(file_stats) = &file.statistics { + project_schema_onto_datasrc_statistics( + file_stats, + file_schema, + &target_schema, + )? + } else { + // doesn't have file stats + Arc::new(Statistics::new_unknown(&target_schema)) + }; + Ok::, DataFusionError>(statistics) + }) + .process_results(|stats| { + Arc::new(merge_stats_collection( + stats.into_iter(), + &target_schema, + )) + }) + }) + .try_collect()?; + + Ok(per_partition) + } +} + +impl PartitionStatistics for UnionExec { + fn statistics_by_partition(&self) -> Result { + preserve_partitioning_no_projection(self) + } +} + +impl PartitionStatistics for CoalesceBatchesExec { + fn statistics_by_partition(&self) -> Result { + preserve_partitioning_no_projection_apply_fetch(self, self.fetch(), 0) + } +} + +impl PartitionStatistics for LocalLimitExec { + fn statistics_by_partition(&self) -> Result { + preserve_partitioning_no_projection_apply_fetch(self, Some(self.fetch()), 0) + } +} + +impl PartitionStatistics for GlobalLimitExec { + fn statistics_by_partition(&self) -> Result { + preserve_partitioning_no_projection_apply_fetch(self, self.fetch(), self.skip()) + } +} + +impl PartitionStatistics for FilterExec { + fn statistics_by_partition(&self) -> Result { + preserve_partitioning_with_schema_projection_apply_filter( + self, + self.projection(), + self.predicate(), + self.default_selectivity(), + ) + } +} + +impl PartitionStatistics for ProjectionExec { + fn statistics_by_partition(&self) -> Result { + // use the specific `proj_exec_stats` to project schema on every incoming partition + let partition_cnt = partition_count(self); + + let all_partition_stats = self.children().into_iter().try_fold( + Vec::with_capacity(partition_cnt), + |mut acc, child| { + let child_stats = statistics_by_partition(child.as_ref())?; + + let child_stats_with_project_exec_projected = + child_stats.into_iter().map(|stats| { + proj_exec_stats( + Arc::unwrap_or_clone(stats), + self.expr().iter(), + &self.schema(), + ) + }); + + acc.extend(child_stats_with_project_exec_projected); + Ok::(acc) + }, + )?; + + Ok(all_partition_stats) + } +} + +impl PartitionStatistics for SortPreservingMergeExec { + fn statistics_by_partition(&self) -> Result { + merge_partitions_no_projection_apply_fetch(self, self.fetch(), 0) + } +} + +// TODO: once this is implemented +// impl PartitionStatistics for ProgressiveEvalExec { +// fn statistics_by_partition(&self) -> Result { +// merge_partitions_no_projection_apply_fetch(self, self.fetch(), 0) +// } +// } + +impl PartitionStatistics for CoalescePartitionsExec { + fn statistics_by_partition(&self) -> Result { + merge_partitions_no_projection(self) + } +} + +impl PartitionStatistics for SortExec { + fn statistics_by_partition(&self) -> Result { + if self.preserve_partitioning() { + preserve_partitioning_no_projection_apply_fetch(self, self.fetch(), 0) + } else { + merge_partitions_no_projection_apply_fetch(self, self.fetch(), 0) + } + } +} + +impl PartitionStatistics for RepartitionExec { + fn statistics_by_partition(&self) -> Result { + // takes N input partitions, returning M output partitions. + + // the statistics are "merged" into the same value (because any input can go to any output) + let mut merged = merge_partitions_no_projection(self)?; + let merged_single = merged.pop().expect("should have a single merged statistic"); + let merged_single = Arc::unwrap_or_clone(merged_single); + + // then the merged stat is turned to inexact (since unknown division across output partitions) + let inexact_merged = Arc::new(Statistics { + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, + column_statistics: make_column_statistics_inexact( + merged_single.column_statistics, + ), + }); + + // finally, all output partitions have the same merged stat + Ok(vec![inexact_merged; partition_count(self)]) + } +} + +/// Handle downcasting of the `dyn ExecutionPlan` to the specific execution nodes which +/// have implemented [`PartitionStatistics`]. +pub fn statistics_by_partition( + plan: &dyn ExecutionPlan, +) -> Result { + if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + } else if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + } else if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + } else if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + } else if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + } else if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + } else if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + } else if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + } else if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + } else if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + // } else if let Some(exec) = plan.as_any().downcast_ref::() { + // exec.statistics_by_partition() + } else if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + } else if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + } else if let Some(exec) = plan.as_any().downcast_ref::() { + exec.statistics_by_partition() + } else { + /* These include, but not limited to, the following operators used in our example plans: + WindowAggExec + SymmetricHashJoinExec + HashJoinExec + NestedLoopJoinExec + ValuesExec + AggregateExec + */ + Ok(unknown_statistics_by_partition(plan)) + } +} + +/// Provide unknown/absent statistics for all partitions in the plan. +fn unknown_statistics_by_partition(plan: &dyn ExecutionPlan) -> PartitionedStatistics { + let data = Arc::new(Statistics::new_unknown(&plan.schema())); + vec![data; partition_count(plan)] +} + +/// Preserve partitioning from plan inputs. +/// +/// This does not perform any schema projection. +fn preserve_partitioning_no_projection( + plan: &dyn ExecutionPlan, +) -> Result { + preserve_partitioning_with_schema_projection(plan, None) +} + +/// Preserve partitioning from plan inputs. +/// Apply fetch variables (fetch/skip) to modify the [`Statistics::num_rows`]. +/// +/// This does not perform any schema projection. +fn preserve_partitioning_no_projection_apply_fetch( + plan: &dyn ExecutionPlan, + fetch: Option, + skip: usize, +) -> Result { + let partition_cnt = partition_count(plan); + let all_partition_stats = plan.children().into_iter().try_fold( + Vec::with_capacity(partition_cnt), + |mut acc, child| { + let child_stats = statistics_by_partition(child.as_ref())?; + + let child_stats_with_fetch = child_stats + .into_iter() + .map(|stats| apply_fetch(stats, fetch, skip)); + + acc.extend(child_stats_with_fetch); + Ok::(acc) + }, + )?; + + Ok(all_partition_stats) +} + +/// Preserve partitioning from plan inputs. +/// Performs a schema projection to modify the [`Statistics::column_statistics`]. +/// +/// A schema projection is only required if either a subset of input fields are projected +/// into the plan output (e.g. it has a `self.projection`), or we need to add column_statistics +/// for a chunk order column. +fn preserve_partitioning_with_schema_projection( + plan: &dyn ExecutionPlan, + subset_selected: Option<&Vec>, +) -> Result { + let target_schema = plan.schema(); + let mut all_partition_stats = Vec::with_capacity(partition_count(plan)); + + for child in plan.children() { + let child_stats = statistics_by_partition(child.as_ref())?; + + child_stats + .into_iter() + .map(|stats| { + if let Some(proj_idx) = subset_selected { + // apply a schema projection + project_select_subset_of_column_statistics( + &stats, + &child.schema(), + proj_idx, + &target_schema, + ) + } else { + Ok(stats) + } + }) + .process_results(|iter| all_partition_stats.extend(iter))?; + } + + Ok(all_partition_stats) +} + +/// Preserve partitioning from plan inputs. +/// Apply filter variables to modify the num_rows, total_byte_size, column_statistics in [`Statistics`]. +/// +/// Then performs a schema projection to modify the [`Statistics::column_statistics`]. +fn preserve_partitioning_with_schema_projection_apply_filter( + plan: &dyn ExecutionPlan, + project: Option<&Vec>, + predicate: &Arc, + default_selectivity: u8, +) -> Result { + let target_schema = plan.schema(); + let mut all_partition_stats = Vec::with_capacity(partition_count(plan)); + + for child in plan.children() { + let child_stats = statistics_by_partition(child.as_ref())?; + + child_stats + .into_iter() + .map(|stats| { + // apply filter first on input child + apply_filter( + Arc::unwrap_or_clone(stats), + &child.schema(), + predicate, + default_selectivity, + ) + }) + // then apply schema projection on output + .map_ok(|stats| { + if let Some(proj_idx) = project { + project_select_subset_of_column_statistics( + &stats, + &child.schema(), + proj_idx, + &target_schema, + ) + } else { + Ok(Arc::new(stats)) + } + }) + .flatten_ok() + .process_results(|iter| all_partition_stats.extend(iter))?; + } + + Ok(all_partition_stats) +} + +/// Merge partition stats across plan inputs. +/// +/// This does not perform any schema projection. +fn merge_partitions_no_projection( + plan: &dyn ExecutionPlan, +) -> Result { + merge_partitions_no_projection_apply_fetch(plan, None, 0) +} + +/// Merge partition stats across plan inputs. +/// Apply fetch variables (fetch/skip) to modify the [`Statistics::num_rows`]. +/// +/// This does not perform any schema projection. +fn merge_partitions_no_projection_apply_fetch( + plan: &dyn ExecutionPlan, + fetch: Option, + skip: usize, +) -> Result { + let merged_partition_stats = plan + .children() + .into_iter() + .map(|child| { + let child_stats = statistics_by_partition(child.as_ref())?; + + // merge stats for all partitions in each child + let merged_child_partitions = if fetch.is_some() || skip > 0 { + let child_stats_with_fetch = child_stats + .into_iter() + .map(|stats| apply_fetch(stats, fetch, skip)); + + merge_stats_collection(child_stats_with_fetch, &plan.schema()) + } else { + merge_stats_collection(child_stats.into_iter(), &plan.schema()) + }; + + Ok::(merged_child_partitions) + }) + .process_results(|stats| { + // merge stats across children + Arc::new(merge_stats_collection(stats, &plan.schema())) + })?; + + Ok(vec![merged_partition_stats]) +} + +#[cfg(test)] +mod test { + use std::fmt::{Display, Formatter}; + + use crate::progressive_evaluation::util::test_utils::{ + coalesce_exec, crossjoin_exec, file_scan_config, filter_exec, limit_exec, + parquet_exec_with_sort_with_statistics, proj_exec, repartition_exec, + single_column_schema, sort_exec, spm_exec, union_exec, PartitionedFilesAndRanges, + SortKeyRange, + }; + + use super::*; + use arrow::datatypes::{DataType, Field}; + use datafusion_common::{ColumnStatistics, DataFusionError}; + use datafusion_expr::Operator; + use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; + use datafusion_physical_plan::{ + displayable, + expressions::{col, lit, BinaryExpr, IsNullExpr, NoOp}, + union::InterleaveExec, + Partitioning, + }; + + use insta::assert_snapshot; + use itertools::Itertools; + + /// For running test cases on the [`statistics_by_partition`]. + #[derive(Debug)] + struct TestCase<'a> { + /// Input place to test. + input_plan: &'a Arc, + /// Column to extract. + col_name: &'a str, + /// Expected column statistics, per partition. + expected_ranges_per_partition: Option<&'a [&'a SortKeyRange]>, + /// Actual results per partition, populated after [`TestCase::run`]. + result_per_partition: PartitionedStatistics, + } + + impl<'a> TestCase<'a> { + fn new( + input_plan: &'a Arc, + col_name: &'a str, + expected_ranges_per_partition: Option<&'a [&'a SortKeyRange]>, + ) -> Self { + Self { + input_plan, + col_name, + expected_ranges_per_partition, + result_per_partition: vec![], + } + } + + /// Run the test cases, and populate the results in [`TestCase::result_per_partition`]. + fn run(mut self) -> Result { + let partition_cnt = self.input_partition_cnt(); + + if let Some(per_partition) = &self.expected_ranges_per_partition { + assert_eq!( + per_partition.len(), + partition_cnt, + "failure in test setup, the count of expected ranges should equal the partition count" + ); + }; + + // run test case with PartitionStatisitics implementations + self.result_per_partition = + statistics_by_partition(self.input_plan.as_ref())?; + + Ok(self) + } + + fn input_partition_cnt(&self) -> usize { + self.input_plan + .properties() + .output_partitioning() + .partition_count() + } + + /// Resultant [`ColumnStatistics`] per partition, extracted from the schema [`Statistics`], + /// for the test column. + /// + /// If the schema does not have the resultant test column (e.g. the output plan projection is `select another-col`) + /// Then the result is None. + fn results_for_col_name(&self) -> Vec> { + if let Ok(col_idx) = self.input_plan.schema().index_of(self.col_name) { + self.result_per_partition + .iter() + .map(|stats| Some(stats.column_statistics[col_idx].clone())) + .collect_vec() + } else { + vec![None; self.input_partition_cnt()] + } + } + } + + impl Display for TestCase<'_> { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + let displayable_plan = displayable(self.input_plan.as_ref()).indent(false); + writeln!(f, "{}", displayable_plan)?; + + writeln!(f, "Expected column statistics per partition:")?; + for partition in 0..self.input_partition_cnt() { + if let Some(expected_per_partition) = self.expected_ranges_per_partition { + writeln!( + f, + " partition {:?}: {}", + partition, expected_per_partition[partition] + )?; + } else { + writeln!(f, " partition {:?}: None", partition)?; + } + } + + writeln!(f, "\nActual column statistics per partition:")?; + for (partition, actual_stats) in + self.results_for_col_name().iter().enumerate() + { + writeln!( + f, + " partition {:?}: {}", + partition, + str_column_stats(actual_stats) + )?; + } + + Ok(()) + } + } + + /// Provide the [`ColumnStatistics`] as a string for insta. + fn str_column_stats(stats: &Option) -> String { + let Some(stats) = stats else { + return "None".into(); + }; + if matches!(stats.min_value, Precision::Absent) + && matches!(stats.max_value, Precision::Absent) + { + return "None".into(); + } + + if stats.null_count.get_value().cloned().unwrap_or_default() > 0 { + format!( + "({:?})->({:?}) null_count={}", + stats.min_value.get_value().unwrap(), + stats.max_value.get_value().unwrap(), + stats.null_count.get_value().unwrap() + ) + } else { + format!( + "({:?})->({:?})", + stats.min_value.get_value().unwrap(), + stats.max_value.get_value().unwrap() + ) + } + } + + /// Plan nodes which generate the original statistics (a.k.a. the data sources). + #[test] + fn test_handles_datasources() -> Result<(), DataFusionError> { + let col_name = "a"; + let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new( + col(col_name, &single_column_schema())?, + Default::default(), + )]); + let ranges_per_partition = &[ + &SortKeyRange { + min: Some(1000), + max: Some(2000), + null_count: 0, + }, + &SortKeyRange { + min: Some(2001), + max: Some(3000), + null_count: 0, + }, + &SortKeyRange { + min: Some(3001), + max: Some(3500), + null_count: 0, + }, + ]; + + /* Test Case: parquet */ + let plan = parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + ranges_per_partition, + ); + let test_case = TestCase::new(&plan, col_name, Some(ranges_per_partition)); + assert_snapshot!( + test_case.run()?, + @r" + DataSourceExec: file_groups={3 groups: [[0.parquet], [1.parquet], [2.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(2000)) + partition 1: (Some(2001))->(Some(3000)) + partition 2: (Some(3001))->(Some(3500)) + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(2000)) + partition 1: (Int32(2001))->(Int32(3000)) + partition 2: (Int32(3001))->(Int32(3500)) + "); + + /* Test Case: empty exec */ + // note: empty exec always has only 1 partition, and has absent stats (except for 0 num_rows) + let plan = Arc::new(EmptyExec::new(single_column_schema())) as _; + let test_case = TestCase::new(&plan, col_name, None); + let result = test_case.run()?; + assert_snapshot!( + result, + @r" + EmptyExec + + Expected column statistics per partition: + partition 0: None + + Actual column statistics per partition: + partition 0: None + "); + assert_eq!( + result.result_per_partition[0].num_rows.get_value(), + Some(&0), + "empty exec should have zero rows" + ); + + /* Test Case: placeholder row */ + // note: placeholder row is 1 row with null values for all columns + let plan = + Arc::new(PlaceholderRowExec::new(single_column_schema()).with_partitions(2)) + as _; + let test_case = TestCase::new(&plan, col_name, None); + let result = test_case.run()?; + assert_snapshot!( + result, + @r" + PlaceholderRowExec + + Expected column statistics per partition: + partition 0: None + partition 1: None + + Actual column statistics per partition: + partition 0: None + partition 1: None + "); + assert_eq!( + result.result_per_partition.len(), + 2, + "should have stats for 2 partitions" + ); + let [p0, p1] = &result.result_per_partition[..] else { + unreachable!() + }; + assert_eq!( + p0.num_rows.get_value(), + Some(1).as_ref(), + "should have only 1 row" + ); + assert_eq!( + p0.column_statistics[0].null_count.get_value(), + Some(1).as_ref(), + "should be a null value" + ); + assert_eq!( + p1.num_rows.get_value(), + Some(1).as_ref(), + "should have only 1 row" + ); + assert_eq!( + p1.column_statistics[0].null_count.get_value(), + Some(1).as_ref(), + "should be a null value" + ); + + Ok(()) + } + + /// Plan nodes which pass through the partitions without impacting range. + #[test] + fn test_handles_partition_pass_thru() -> Result<(), DataFusionError> { + let col_name = "a"; + let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new( + col(col_name, &single_column_schema())?, + Default::default(), + )]); + let ranges_per_partition = &[ + &SortKeyRange { + min: Some(1000), + max: Some(2000), + null_count: 0, + }, + &SortKeyRange { + min: Some(2001), + max: Some(3000), + null_count: 0, + }, + &SortKeyRange { + min: Some(3001), + max: Some(3500), + null_count: 0, + }, + ]; + + /* Test Case: union */ + let left_plan = parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + &ranges_per_partition[0..1], + ); + let right_plan = parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + &ranges_per_partition[1..], + ); + let union = union_exec(vec![left_plan, right_plan]); + let test_case = TestCase::new(&union, col_name, Some(ranges_per_partition)); + assert_snapshot!( + test_case.run()?, + @r" + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + DataSourceExec: file_groups={2 groups: [[0.parquet], [1.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(2000)) + partition 1: (Some(2001))->(Some(3000)) + partition 2: (Some(3001))->(Some(3500)) + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(2000)) + partition 1: (Int32(2001))->(Int32(3000)) + partition 2: (Int32(3001))->(Int32(3500)) + "); + + /* Test Case: sorts, with preserve partitioning */ + let preserve_partitioning = true; + let input = parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + ranges_per_partition, + ); + let sort = sort_exec(&lex_ordering, &input, preserve_partitioning); + let test_case = TestCase::new(&sort, col_name, Some(ranges_per_partition)); + assert_snapshot!( + test_case.run()?, + @r" + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + DataSourceExec: file_groups={3 groups: [[0.parquet], [1.parquet], [2.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(2000)) + partition 1: (Some(2001))->(Some(3000)) + partition 2: (Some(3001))->(Some(3500)) + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(2000)) + partition 1: (Int32(2001))->(Int32(3000)) + partition 2: (Int32(3001))->(Int32(3500)) + "); + + /* Test Case: coalesce */ + let coalesce = coalesce_exec(&input); + let test_case = TestCase::new(&coalesce, col_name, Some(ranges_per_partition)); + assert_snapshot!( + test_case.run()?, + @r" + CoalesceBatchesExec: target_batch_size=10 + DataSourceExec: file_groups={3 groups: [[0.parquet], [1.parquet], [2.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(2000)) + partition 1: (Some(2001))->(Some(3000)) + partition 2: (Some(3001))->(Some(3500)) + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(2000)) + partition 1: (Int32(2001))->(Int32(3000)) + partition 2: (Int32(3001))->(Int32(3500)) + "); + + /* Test Case: limit */ + let limit = limit_exec(&input, 2); + let test_case = TestCase::new(&limit, col_name, Some(ranges_per_partition)); + assert_snapshot!( + test_case.run()?, + @r" + LocalLimitExec: fetch=2 + DataSourceExec: file_groups={3 groups: [[0.parquet], [1.parquet], [2.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(2000)) + partition 1: (Some(2001))->(Some(3000)) + partition 2: (Some(3001))->(Some(3500)) + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(2000)) + partition 1: (Int32(2001))->(Int32(3000)) + partition 2: (Int32(3001))->(Int32(3500)) + "); + + /* Test Case: empty projection */ + let proj = proj_exec( + &input, + vec![(col(col_name, &single_column_schema())?, "a".into())], + ); + let test_case = TestCase::new(&proj, col_name, Some(ranges_per_partition)); + assert_snapshot!( + test_case.run()?, + @r" + ProjectionExec: expr=[a@0 as a] + DataSourceExec: file_groups={3 groups: [[0.parquet], [1.parquet], [2.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(2000)) + partition 1: (Some(2001))->(Some(3000)) + partition 2: (Some(3001))->(Some(3500)) + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(2000)) + partition 1: (Int32(2001))->(Int32(3000)) + partition 2: (Int32(3001))->(Int32(3500)) + "); + + /* Test Case: pass thru projection */ + let pass_thru_single_col = col("a", &single_column_schema())?; + let proj = proj_exec( + &input, + vec![(Arc::clone(&pass_thru_single_col), "a".into())], + ); + let test_case = TestCase::new(&proj, col_name, Some(ranges_per_partition)); + assert_snapshot!( + test_case.run()?, + @r" + ProjectionExec: expr=[a@0 as a] + DataSourceExec: file_groups={3 groups: [[0.parquet], [1.parquet], [2.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(2000)) + partition 1: (Some(2001))->(Some(3000)) + partition 2: (Some(3001))->(Some(3500)) + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(2000)) + partition 1: (Int32(2001))->(Int32(3000)) + partition 2: (Int32(3001))->(Int32(3500)) + "); + + /* Test Case: projection that modifies -> will not pass thru*/ + let col_plus_2 = Arc::new(BinaryExpr::new( + pass_thru_single_col, + Operator::Plus, + lit(2), + )); + let proj = proj_exec(&input, vec![(col_plus_2, "foo".into())]); + let test_case = TestCase::new(&proj, col_name, Some(ranges_per_partition)); + assert_snapshot!( + test_case.run()?, + @r" + ProjectionExec: expr=[a@0 + 2 as foo] + DataSourceExec: file_groups={3 groups: [[0.parquet], [1.parquet], [2.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(2000)) + partition 1: (Some(2001))->(Some(3000)) + partition 2: (Some(3001))->(Some(3500)) + + Actual column statistics per partition: + partition 0: None + partition 1: None + partition 2: None + "); + + /* Test Case: filter (for now, we don't narrow the range further) */ + let filter = + filter_exec(&input, Arc::new(IsNullExpr::new(Arc::new(NoOp::new())))); + let test_case = TestCase::new(&filter, col_name, Some(ranges_per_partition)); + assert_snapshot!( + test_case.run()?, + @r" + FilterExec: NoOp IS NULL + DataSourceExec: file_groups={3 groups: [[0.parquet], [1.parquet], [2.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(2000)) + partition 1: (Some(2001))->(Some(3000)) + partition 2: (Some(3001))->(Some(3500)) + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(2000)) + partition 1: (Int32(2001))->(Int32(3000)) + partition 2: (Int32(3001))->(Int32(3500)) + "); + + Ok(()) + } + + /// Plan nodes which merge the partitions into a single output partition, producing a range + /// predictable from the merged input partitions. + #[test] + fn test_handles_partition_merging() -> Result<(), DataFusionError> { + let col_name = "a"; + let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new( + col(col_name, &single_column_schema())?, + Default::default(), + )]); + let ranges_per_partition = &[ + &SortKeyRange { + min: Some(1000), + max: Some(2000), + null_count: 0, + }, + &SortKeyRange { + min: Some(2001), + max: Some(3000), + null_count: 0, + }, + &SortKeyRange { + min: Some(3001), + max: Some(3500), + null_count: 0, + }, + ]; + + // Expected result from all test cases. + let partitioned_input = parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + ranges_per_partition, + ); + let expect_merged = &[&SortKeyRange { + min: Some(1000), + max: Some(3500), + null_count: 0, + }]; + + /* Test Case: SPM */ + let spm = spm_exec(&partitioned_input, &lex_ordering); + let test_case = TestCase::new(&spm, col_name, Some(expect_merged)); + assert_snapshot!( + test_case.run()?, + @r" + SortPreservingMergeExec: [a@0 ASC] + DataSourceExec: file_groups={3 groups: [[0.parquet], [1.parquet], [2.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(3500)) + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(3500)) + "); + + /* Test Case: sorts, without preserve partitioning */ + let sort = sort_exec(&lex_ordering, &partitioned_input, false); + let test_case = TestCase::new(&sort, col_name, Some(expect_merged)); + assert_snapshot!( + test_case.run()?, + @r" + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={3 groups: [[0.parquet], [1.parquet], [2.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(3500)) + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(3500)) + "); + + Ok(()) + } + + /// Plan nodes which has N input partitions and M output partitions, where N may not equal M + /// and partitions are reshuffled. + #[test] + fn test_handles_repartitioning() -> Result<(), DataFusionError> { + let col_name = "a"; + let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new( + col(col_name, &single_column_schema())?, + Default::default(), + )]); + let ranges_per_partition = &[ + &SortKeyRange { + min: Some(1000), + max: Some(2000), + null_count: 0, + }, + &SortKeyRange { + min: Some(2001), + max: Some(3000), + null_count: 0, + }, + &SortKeyRange { + min: Some(3001), + max: Some(3500), + null_count: 0, + }, + ]; + + // Expected result from all test cases. + let expect_merged = &[&SortKeyRange { + min: Some(1000), + max: Some(3500), + null_count: 0, + }]; + + /* Test Case: Repartitioning */ + let partitioned_input = parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + ranges_per_partition, + ); + let partitioning = Partitioning::Hash(vec![], 4); + let repartition = repartition_exec(&partitioned_input, partitioning); + // expect all 4 hashed partitions to potentially cover the same range + let expected = std::iter::repeat_n(expect_merged[0], 4).collect_vec(); + let test_case = TestCase::new(&repartition, col_name, Some(&expected)); + assert_snapshot!( + test_case.run()?, + @r" + RepartitionExec: partitioning=Hash([], 4), input_partitions=3 + DataSourceExec: file_groups={3 groups: [[0.parquet], [1.parquet], [2.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(3500)) + partition 1: (Some(1000))->(Some(3500)) + partition 2: (Some(1000))->(Some(3500)) + partition 3: (Some(1000))->(Some(3500)) + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(3500)) + partition 1: (Int32(1000))->(Int32(3500)) + partition 2: (Int32(1000))->(Int32(3500)) + partition 3: (Int32(1000))->(Int32(3500)) + "); + + Ok(()) + } + + fn build_interleave_plan( + lex_ordering: LexOrdering, + ranges_per_partition: &[&SortKeyRange], + ) -> Result, DataFusionError> { + let partitioning = Partitioning::Hash(vec![], 4); + let left_plan = repartition_exec( + &parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + &ranges_per_partition[0..1], + ), + partitioning.clone(), + ); + let right_plan = repartition_exec( + &parquet_exec_with_sort_with_statistics( + vec![lex_ordering], + &ranges_per_partition[1..], + ), + partitioning, + ); + Ok(Arc::new(InterleaveExec::try_new(vec![ + left_plan, right_plan, + ])?)) + } + + /// Plan nodes which short circuit the statistics_by_partition, returning None (or absent) statistics. + #[test] + fn test_returns_none() -> Result<(), DataFusionError> { + let col_name = "a"; + let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new( + col(col_name, &single_column_schema())?, + Default::default(), + )]); + let ranges_per_partition = &[ + &SortKeyRange { + min: Some(1000), + max: Some(2000), + null_count: 0, + }, + &SortKeyRange { + min: Some(2001), + max: Some(3000), + null_count: 0, + }, + &SortKeyRange { + min: Some(3001), + max: Some(3500), + null_count: 0, + }, + ]; + + /* Test Case: Joins */ + let left_plan = parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + &ranges_per_partition[0..1], + ); + let right_plan = parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + &ranges_per_partition[1..], + ); + let crossjoin = crossjoin_exec(&left_plan, &right_plan); + let test_case = TestCase::new(&crossjoin, col_name, None); + assert_snapshot!( + test_case.run()?, + @r" + CrossJoinExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + DataSourceExec: file_groups={2 groups: [[0.parquet], [1.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: None + partition 1: None + + Actual column statistics per partition: + partition 0: None + partition 1: None + "); + + /* Test Case: interleave */ + let interleave = + build_interleave_plan(lex_ordering.clone(), ranges_per_partition)?; + let test_case = TestCase::new(&interleave, col_name, None); + assert_snapshot!( + test_case.run()?, + @r" + InterleaveExec + RepartitionExec: partitioning=Hash([], 4), input_partitions=1 + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + RepartitionExec: partitioning=Hash([], 4), input_partitions=2 + DataSourceExec: file_groups={2 groups: [[0.parquet], [1.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: None + partition 1: None + partition 2: None + partition 3: None + + Actual column statistics per partition: + partition 0: None + partition 1: None + partition 2: None + partition 3: None + "); + + /* Test Case: None will override later merges with partitions having stats */ + // (because None means cannot determine all of the subplan stats) + let partitioned_input = parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + ranges_per_partition, + ); + let spm = spm_exec( + &union_exec(vec![partitioned_input, interleave]), + &lex_ordering, + ); + let test_case = TestCase::new(&spm, col_name, None); + assert_snapshot!( + test_case.run()?, + @r" + SortPreservingMergeExec: [a@0 ASC] + UnionExec + DataSourceExec: file_groups={3 groups: [[0.parquet], [1.parquet], [2.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + InterleaveExec + RepartitionExec: partitioning=Hash([], 4), input_partitions=1 + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + RepartitionExec: partitioning=Hash([], 4), input_partitions=2 + DataSourceExec: file_groups={2 groups: [[0.parquet], [1.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: None + + Actual column statistics per partition: + partition 0: None + "); + + Ok(()) + } + + /// How null counts are handled. + #[test] + fn test_null_counts() -> Result<(), DataFusionError> { + let col_name = "a"; + let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new( + col(col_name, &single_column_schema())?, + Default::default(), + )]); + let ranges_per_partition = &[ + &SortKeyRange { + min: Some(1000), + max: Some(2000), + null_count: 1, + }, + &SortKeyRange { + min: Some(2001), + max: Some(3000), + null_count: 4, + }, + &SortKeyRange { + min: Some(3001), + max: Some(3500), + null_count: 3, + }, + ]; + + /* Test Case: keeps null counts separate with pass thru */ + let left_plan = parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + &ranges_per_partition[0..1], + ); + let right_plan = parquet_exec_with_sort_with_statistics( + vec![lex_ordering.clone()], + &ranges_per_partition[1..], + ); + let union = union_exec(vec![left_plan, right_plan]); + let test_case = TestCase::new(&union, col_name, Some(ranges_per_partition)); + assert_snapshot!( + test_case.run()?, + @r" + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + DataSourceExec: file_groups={2 groups: [[0.parquet], [1.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(2000)) null_count=1 + partition 1: (Some(2001))->(Some(3000)) null_count=4 + partition 2: (Some(3001))->(Some(3500)) null_count=3 + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(2000)) null_count=1 + partition 1: (Int32(2001))->(Int32(3000)) null_count=4 + partition 2: (Int32(3001))->(Int32(3500)) null_count=3 + "); + + /* Test Case: merges null counts */ + let spm = spm_exec(&union, &lex_ordering); + let expect_merged = &[&SortKeyRange { + min: Some(1000), + max: Some(3500), + null_count: 8, + }]; + let test_case = TestCase::new(&spm, col_name, Some(expect_merged)); + assert_snapshot!( + test_case.run()?, + @r" + SortPreservingMergeExec: [a@0 ASC] + UnionExec + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + DataSourceExec: file_groups={2 groups: [[0.parquet], [1.parquet]]}, projection=[a], output_ordering=[a@0 ASC], file_type=parquet + + Expected column statistics per partition: + partition 0: (Some(1000))->(Some(3500)) null_count=8 + + Actual column statistics per partition: + partition 0: (Int32(1000))->(Int32(3500)) null_count=8 + "); + + Ok(()) + } + + /// Test we are using the proper file schema + #[test] + fn test_file_group() -> Result<(), DataFusionError> { + // target column + let col_name = "C"; + + // File schema, vs plan schema. + let file_schema = Arc::new(arrow::datatypes::Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new(col_name, DataType::Int64, true), + ])); + let plan_schema = Arc::new(arrow::datatypes::Schema::new(vec![Field::new( + col_name, + DataType::Int64, + true, + )])); + + // File scan config uses the file schema and ALL columns in file. + let ranges_for_file_0 = vec![ + // col a + SortKeyRange { + min: Some(20), + max: Some(30), + null_count: 1, + }, + // col b + SortKeyRange { + min: Some(2000), + max: Some(3000), + null_count: 1, + }, + // col C (our tested key to extract) + SortKeyRange { + min: Some(200_000), + max: Some(300_000), + null_count: 1, + }, + ]; + let ranges_for_file_1 = vec![ + // col a + SortKeyRange { + min: Some(30), + max: Some(40), + null_count: 1, + }, + // col b + SortKeyRange { + min: Some(1000), + max: Some(2000), + null_count: 1, + }, + // col C (our tested key to extract) + SortKeyRange { + min: Some(500_000), + max: Some(700_000), + null_count: 1, + }, + ]; + let lex_ordering_on_c = LexOrdering::new(vec![PhysicalSortExpr::new( + col(col_name, &file_schema)?, // file_schema + Default::default(), + )]); + let multiple_column_key_ranges_per_file = PartitionedFilesAndRanges { + per_file: vec![ranges_for_file_0.clone(), ranges_for_file_1.clone()], + }; + #[expect(deprecated)] + let filegroups_config = file_scan_config( + &file_schema, + vec![lex_ordering_on_c], + multiple_column_key_ranges_per_file, + ) + .with_projection(Some(vec![2])); + + // use a plan with only col C + let parquet_exec = DataSourceExec::from_data_source(filegroups_config) as _; + let projected = col(col_name, &plan_schema)?; // plan_schema + let plan = proj_exec(&parquet_exec, vec![(projected, "C".into())]); + insta::assert_snapshot!( + displayable(plan.as_ref()).indent(true), + @r" + ProjectionExec: expr=[C@0 as C] + DataSourceExec: file_groups={2 groups: [[0.parquet], [1.parquet]]}, projection=[C], output_ordering=[C@0 ASC], file_type=parquet + ", + ); + + // Test Case: show that plan schema is different from file schema + assert_ne!( + &plan_schema, &file_schema, + "plan and file schema are not equivalent" + ); + assert_eq!( + plan.schema().fields().len(), + 1, + "plan schema should have only 1 field" + ); + assert_eq!( + plan.schema(), + plan_schema, + "plan schema should only have col C" + ); + let Some(parquet_exec) = + plan.children()[0].as_any().downcast_ref::() + else { + unreachable!() + }; + assert_eq!( + parquet_exec.schema(), + plan_schema, + "parquet exec plan schema should only have col C" + ); + + /* Test Case: the statistics_by_partition will still extract the proper file_stats for col C */ + let actual = statistics_by_partition(plan.as_ref())?; + let [actual_partition_0, actual_partition_1] = &actual[..] else { + panic!("should have stats for 2 partitions"); + }; + assert_eq!( + actual_partition_0.column_statistics.len(), + 1, + "should have only 1 column for the ProjectExec C@0" + ); + assert_eq!( + actual_partition_1.column_statistics.len(), + 1, + "should have only 1 column for the ProjectExec C@0" + ); + + // partition 0 == ranges_for_file_0 + let expected = ranges_for_file_0[2]; + assert_eq!(actual_partition_0.column_statistics[0], expected.into()); + + // partition 1 == ranges_for_file_1 + let expected = ranges_for_file_1[2]; + assert_eq!(actual_partition_1.column_statistics[0], expected.into()); + + Ok(()) + } + + #[test] + fn test_extracts_multiple_cols_at_once() -> Result<(), DataFusionError> { + // plan with multiple fields + let mut fields = vec![ + Field::new("_not_used_file_col", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Int64, true), + ]; + let file_schema = Arc::new(arrow::datatypes::Schema::new(fields.clone())); + let plan_schema = Arc::new(arrow::datatypes::Schema::new(fields.split_off(1))); + + // File scan config uses the file schema and ALL columns in file. + let ranges_for_file_0 = vec![ + // col _not_used_file_col + SortKeyRange { + min: Some(20), + max: Some(30), + null_count: 1, + }, + // col b + SortKeyRange { + min: Some(2000), + max: Some(3000), + null_count: 1, + }, + // col c + SortKeyRange { + min: Some(200_000), + max: Some(300_000), + null_count: 1, + }, + ]; + let ranges_for_file_1 = vec![ + // col _not_used_file_col + SortKeyRange { + min: Some(30), + max: Some(40), + null_count: 1, + }, + // col b + SortKeyRange { + min: Some(1000), + max: Some(2000), + null_count: 1, + }, + // col c + SortKeyRange { + min: Some(500_000), + max: Some(700_000), + null_count: 1, + }, + ]; + + // make file config, using file schema + let lex_ordering_on_c = LexOrdering::new(vec![ + PhysicalSortExpr::new(col("c", &file_schema)?, Default::default()), + PhysicalSortExpr::new(col("b", &file_schema)?, Default::default()), + ]); + let multiple_column_key_ranges_per_file = PartitionedFilesAndRanges { + per_file: vec![ranges_for_file_0.clone(), ranges_for_file_1.clone()], + }; + #[expect(deprecated)] + let filegroups_config = file_scan_config( + &file_schema, // file_schema + vec![lex_ordering_on_c], + multiple_column_key_ranges_per_file, + ) + .with_projection(Some(vec![1, 2])); + + // make plan config, using a plan with only cols b & c + let parquet_exec = DataSourceExec::from_data_source(filegroups_config) as _; + let proj_c = col("c", &plan_schema)?; // plan_schema + let proj_b = col("b", &plan_schema)?; // plan_schema + // plan reverses the 2 cols, c then b + let plan = proj_exec( + &parquet_exec, + vec![(proj_c, "c".into()), (proj_b, "b".into())], + ); + insta::assert_snapshot!( + displayable(plan.as_ref()).indent(true), + @r" + ProjectionExec: expr=[c@1 as c, b@0 as b] + DataSourceExec: file_groups={2 groups: [[0.parquet], [1.parquet]]}, projection=[b, c], output_ordering=[c@1 ASC, b@0 ASC], file_type=parquet + ", + ); + + /* Test Case: the statistics_by_partition will still extract the proper file_stats for both cols c and b */ + let actual = statistics_by_partition(plan.as_ref())?; + let [actual_partition_0, actual_partition_1] = &actual[..] else { + panic!("should have stats for 2 partitions"); + }; + + // partition 0 == ranges_for_file_0 + // use cols c then b, in reverse order for the projection [2..=1] + let expected: Vec = ranges_for_file_0[1..=2] + .iter() + .rev() + .map(|sort_range| (*sort_range).into()) + .collect_vec(); + assert_eq!( + &expected, + &[ + // col c, partition 0 + ColumnStatistics { + null_count: Precision::Exact(1), + min_value: Precision::Exact(ScalarValue::Int32(Some(200_000))), + max_value: Precision::Exact(ScalarValue::Int32(Some(300_000))), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + }, + // col b, partition 0 + ColumnStatistics { + null_count: Precision::Exact(1), + min_value: Precision::Exact(ScalarValue::Int32(Some(2000))), + max_value: Precision::Exact(ScalarValue::Int32(Some(3000))), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + }, + ] + ); + assert_eq!(actual_partition_0.column_statistics, expected); + + // partition 1 == ranges_for_file_1 + // use cols c then b, in reverse order for the projection [2..=1] + let expected: Vec = ranges_for_file_1[1..=2] + .iter() + .rev() + .map(|sort_range| (*sort_range).into()) + .collect_vec(); + assert_eq!( + &expected, + &[ + // col c, partition 1 + ColumnStatistics { + null_count: Precision::Exact(1), + min_value: Precision::Exact(ScalarValue::Int32(Some(500_000))), + max_value: Precision::Exact(ScalarValue::Int32(Some(700_000))), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + }, + // col b, partition 1 + ColumnStatistics { + null_count: Precision::Exact(1), + min_value: Precision::Exact(ScalarValue::Int32(Some(1000))), + max_value: Precision::Exact(ScalarValue::Int32(Some(2000))), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + }, + ] + ); + assert_eq!(actual_partition_1.column_statistics, expected); + + Ok(()) + } + + #[test] + fn test_will_not_extract_for_non_passthru_projections() -> Result<(), DataFusionError> + { + // plan with multiple fields + let file_schema = Arc::new(arrow::datatypes::Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Int64, true), + ])); + let plan_schema = Arc::new(arrow::datatypes::Schema::new(vec![ + Field::new("b", DataType::Int64, true), // will be `a + 1 as b`` + Field::new("c", DataType::Int64, true), + ])); + + // file stats for 3 columns in file_schema + let ranges_for_file = vec![ + // col a + SortKeyRange { + min: Some(20), + max: Some(30), + null_count: 1, + }, + // col b + SortKeyRange { + min: Some(2000), + max: Some(3000), + null_count: 1, + }, + // col c + SortKeyRange { + min: Some(200_000), + max: Some(300_000), + null_count: 1, + }, + ]; + + // make scan plan + let multiple_column_key_ranges_per_file = PartitionedFilesAndRanges { + per_file: vec![ranges_for_file], + }; + #[expect(deprecated)] + let filegroups_config = file_scan_config( + &file_schema, // file_schema + vec![], + multiple_column_key_ranges_per_file, + ) + .with_projection(Some(vec![0, 1, 2])); + let scan = DataSourceExec::from_data_source(filegroups_config) as _; + + // make projection which modifies columns and aliases to an existing columns + let pass_thru_c = col("c", &file_schema)?; + let make_new = Arc::new(BinaryExpr::new( + col("a", &file_schema)?, + Operator::Minus, + lit(1), + )) as _; + let plan = proj_exec( + &scan, + vec![(make_new, "b".into()), (pass_thru_c, "c".into())], + ); + insta::assert_snapshot!( + displayable(plan.as_ref()).indent(true), + @r" + ProjectionExec: expr=[a@0 - 1 as b, c@2 as c] + DataSourceExec: file_groups={1 group: [[0.parquet]]}, projection=[a, b, c], file_type=parquet + ", + ); + assert_eq!( + plan.schema(), + plan_schema, + "should have plan schema with the 2 final projected columns" + ); + + /* Test Case: the statistics_by_partition will still extract the proper file_stats for both cols c and b */ + let actual = statistics_by_partition(plan.as_ref())?; + + // will selectively detect which projection exprs are not a pass thru, and return absent statistics + assert_eq!( + &actual[0].column_statistics, + &[ + // a + 1 as b, partition 0 + ColumnStatistics::new_unknown(), + // col c, partition 0 + ColumnStatistics { + null_count: Precision::Exact(1), + min_value: Precision::Exact(ScalarValue::Int32(Some(200_000))), + max_value: Precision::Exact(ScalarValue::Int32(Some(300_000))), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + }, + ] + ); + + Ok(()) + } +} diff --git a/datafusion/physical-optimizer/src/progressive_evaluation/statistics/fetch.rs b/datafusion/physical-optimizer/src/progressive_evaluation/statistics/fetch.rs new file mode 100644 index 000000000000..4498c16378fa --- /dev/null +++ b/datafusion/physical-optimizer/src/progressive_evaluation/statistics/fetch.rs @@ -0,0 +1,211 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use datafusion_common::{stats::Precision, Statistics}; + +use super::util::make_column_statistics_inexact; + +/// Applies fetch and skip to a num_rows. +/// +/// Fetch is the total num of rows fetched, after the skip is applied. +/// +/// This logic is extracted from [`Statistics::with_fetch`]. Refer to +/// . +pub(super) fn apply_fetch( + stats: Arc, + fetch: Option, + skip: usize, +) -> Arc { + let fetch_applied = match (stats.num_rows, fetch, skip) { + (num_rows, None, 0) => num_rows, + (Precision::Absent, None, _) => Precision::Absent, + + // absent, but can estimate + (Precision::Absent, Some(fetch), _) => Precision::Inexact(fetch), + + // if we are skipping more rows than we have => then 0 output row. + (num_rows, _, skip) + if skip >= *num_rows.get_value().expect("should not be absent") => + { + Precision::Exact(0) + } + + // have exact, return exact + (Precision::Exact(num_rows), fetch, skip) => { + let num_rows_remaining = num_rows.saturating_sub(skip); + let fetched = + std::cmp::min(num_rows_remaining, fetch.unwrap_or(num_rows_remaining)); + Precision::Exact(fetched) + } + + // have inexact, return inexact + (Precision::Inexact(num_rows), fetch, skip) => { + let num_rows_remaining = num_rows.saturating_sub(skip); + let fetched = + std::cmp::min(num_rows_remaining, fetch.unwrap_or(num_rows_remaining)); + Precision::Inexact(fetched) + } + }; + + if stats.num_rows == fetch_applied { + stats + } else { + let mut stats = Arc::unwrap_or_clone(stats); + stats.num_rows = fetch_applied; + + // since the num_rows have to change, also need to modify other stats + stats.total_byte_size = Precision::Absent; + stats.column_statistics = make_column_statistics_inexact(stats.column_statistics); + + Arc::new(stats) + } +} + +#[cfg(test)] +mod tests { + use crate::progressive_evaluation::util::test_utils::single_column_schema; + + use super::*; + + fn build_stats(num_rows: Precision) -> Arc { + let mut stats = Statistics::new_unknown(&single_column_schema()); + stats.num_rows = num_rows; + stats.into() + } + + #[test] + fn test_apply_fetch_to_num_rows() { + /* starting with num_rows = Precision::Absent */ + + // nothing is there + let stats_absent = build_stats(Precision::Absent); + let actual = apply_fetch(Arc::clone(&stats_absent), None, 0); + assert_eq!(actual.num_rows, Precision::Absent, "should still be absent"); + + // absent input, + fetch + let actual = apply_fetch(stats_absent, Some(42), 42); + assert_eq!( + actual.num_rows, + Precision::Inexact(42), + "should return an inexact(fetch) when then input rows are unknown" + ); + + /* starting with num_rows = Precision::Exact */ + + // input rows=100 + let starting_num_rows = Precision::Exact(100); + let stats = build_stats(starting_num_rows); + let actual = apply_fetch(Arc::clone(&stats), None, 0); + assert_eq!( + actual.num_rows, starting_num_rows, + "should return starting input, without fetch nor skip" + ); + + // input rows=100, + skip 25 + let actual = apply_fetch(Arc::clone(&stats), None, 25); + assert_eq!( + actual.num_rows, + Precision::Exact(75), + "should account for skipped rows" + ); + + // input rows=100, + fetch 80 => returns 80 + let actual = apply_fetch(Arc::clone(&stats), Some(80), 0); + assert_eq!( + actual.num_rows, + Precision::Exact(80), + "should account for fetched rows" + ); + + // input rows=100, + fetch 200 => returns 100 + let actual = apply_fetch(Arc::clone(&stats), Some(200), 0); + assert_eq!( + actual.num_rows, + Precision::Exact(100), + "should account for total rows, even with >fetch" + ); + + // input rows=100, + fetch 80 + skip 25 => returns 75 + let actual = apply_fetch(Arc::clone(&stats), Some(80), 25); + assert_eq!( + actual.num_rows, + Precision::Exact(75), + "should be able to return fewer rows, after skip" + ); + + // input rows=100, + fetch 80 + skip 100 => returns 0 + let actual = apply_fetch(Arc::clone(&stats), Some(80), 100); + assert_eq!( + actual.num_rows, + Precision::Exact(0), + "should have no rows to return after skip" + ); + + /* starting with num_rows = Precision::InExact */ + + // input rows=100 + let starting_num_rows = Precision::Inexact(100); + let stats = build_stats(starting_num_rows); + let actual = apply_fetch(Arc::clone(&stats), None, 0); + assert_eq!( + actual.num_rows, starting_num_rows, + "should return starting input, without fetch nor skip" + ); + + // input rows=100, + skip 25 + let actual = apply_fetch(Arc::clone(&stats), None, 25); + assert_eq!( + actual.num_rows, + Precision::Inexact(75), + "should account for skipped rows" + ); + + // input rows=100, + fetch 80 => returns 80 + let actual = apply_fetch(Arc::clone(&stats), Some(80), 0); + assert_eq!( + actual.num_rows, + Precision::Inexact(80), + "should account for fetched rows" + ); + + // input rows=100, + fetch 200 => returns 100 + let actual = apply_fetch(Arc::clone(&stats), Some(200), 0); + assert_eq!( + actual.num_rows, + Precision::Inexact(100), + "should account for total rows, even with >fetch" + ); + + // input rows=100, + fetch 80 + skip 25 => returns 75 + let actual = apply_fetch(Arc::clone(&stats), Some(80), 25); + assert_eq!( + actual.num_rows, + Precision::Inexact(75), + "should be able to return fewer rows, after skip" + ); + + // input rows=100, + fetch 80 + skip 100 => returns 0 + let actual = apply_fetch(Arc::clone(&stats), Some(80), 100); + assert_eq!( + actual.num_rows, + Precision::Exact(0), + "should have no rows to return after skip" + ); + } +} diff --git a/datafusion/physical-optimizer/src/progressive_evaluation/statistics/filter.rs b/datafusion/physical-optimizer/src/progressive_evaluation/statistics/filter.rs new file mode 100644 index 000000000000..5c0f0e5585ed --- /dev/null +++ b/datafusion/physical-optimizer/src/progressive_evaluation/statistics/filter.rs @@ -0,0 +1,68 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use arrow::datatypes::SchemaRef; +use datafusion_common::Statistics; +use datafusion_physical_expr::{ + analyze, intervals::utils::check_support, AnalysisContext, +}; +use datafusion_physical_plan::{filter::collect_new_statistics, PhysicalExpr}; + +/// Calculates [`Statistics`] by applying selectivity (either default, or estimated) to input statistics. +/// +/// This is a (slightly) modified from a private function in datafusion. Refer to: +/// +pub(super) fn apply_filter( + input_stats: Statistics, + input_schema: &SchemaRef, + predicate: &Arc, + default_selectivity: u8, +) -> datafusion_common::Result { + if !check_support(predicate, input_schema) { + let selectivity = default_selectivity as f64 / 100.0; + let mut stats = input_stats.to_inexact(); + stats.num_rows = stats.num_rows.with_estimated_selectivity(selectivity); + stats.total_byte_size = stats + .total_byte_size + .with_estimated_selectivity(selectivity); + return Ok(stats); + } + + 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, + &input_stats.column_statistics, + )?; + + let analysis_ctx = analyze(predicate, input_analysis_ctx, input_schema)?; + + // Estimate (inexact) selectivity of predicate + let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); + let num_rows = num_rows.with_estimated_selectivity(selectivity); + let total_byte_size = total_byte_size.with_estimated_selectivity(selectivity); + + let column_statistics = + collect_new_statistics(&input_stats.column_statistics, analysis_ctx.boundaries); + Ok(Statistics { + num_rows, + total_byte_size, + column_statistics, + }) +} diff --git a/datafusion/physical-optimizer/src/progressive_evaluation/statistics/project_schema.rs b/datafusion/physical-optimizer/src/progressive_evaluation/statistics/project_schema.rs new file mode 100644 index 000000000000..1984c2c32bfe --- /dev/null +++ b/datafusion/physical-optimizer/src/progressive_evaluation/statistics/project_schema.rs @@ -0,0 +1,632 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::{borrow::Borrow, sync::Arc}; + +use arrow::datatypes::SchemaRef; +use datafusion_common::{ + internal_datafusion_err, stats::Precision, ColumnStatistics, Result, ScalarValue, + Statistics, +}; +use datafusion_physical_plan::{expressions::Column, PhysicalExpr}; +use itertools::Itertools; + +use super::util::pretty_fmt_fields; + +/// Takes in a datasource schema and statistics, and projects the scan schema on top. +/// +/// This handles use cases where we may be either: +/// (1) scanning a subset of columns, (b) adding a __chunk_order column, (c) resolving different schema across files. +/// +/// This will error if the src_schema (e.g. file schema) has a field without a corresponding column_statistics. +pub(crate) fn project_schema_onto_datasrc_statistics( + src_statistics: &Statistics, + src_schema: &SchemaRef, + project_schema: &SchemaRef, +) -> Result> { + // (1) scanning a subset of columns (a.k.a. project_schema.fields) + let column_statistics = project_schema + .fields() + .iter() + .map(|field| { + let Ok(col_idx) = src_schema.index_of(field.name()) else { + // adding an empty column (e.g. different parquet files could have slightly different schema). + return Ok(ColumnStatistics { + null_count: src_statistics.num_rows, + max_value: Precision::Exact(ScalarValue::Null), + min_value: Precision::Exact(ScalarValue::Null), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + }); + }; + + let Some(col_stats) = src_statistics.column_statistics.get(col_idx) else { + return Err(internal_datafusion_err!( + "missing column statistics for {}@{}", + field.name(), + col_idx + )); + }; + Ok(col_stats.clone()) + }) + .try_collect()?; + + Ok(Arc::new(Statistics { + num_rows: src_statistics.num_rows, + total_byte_size: src_statistics.total_byte_size, + column_statistics, + })) +} + +/// Determine the output statistics based upon a subset of columns. +/// +/// [`Statistics`] have column_statistics indexed to the plan's schema fields. +/// When the output projected schema has fewer columns than the input schema, +/// (a.k.a. it takes a subselection of columns), +/// then we also need to take a subselection of column stats. +/// +/// This does not have a concept of aliasing, as it requires a `subset_selected` of columns. +/// +/// This will error if the src_schema has a field without a corresponding src column_statistics. +pub(super) fn project_select_subset_of_column_statistics< + T: Borrow + Clone + Into> + std::fmt::Debug, +>( + src_statistics: &T, + src_schema: &SchemaRef, + subset_selected: &Vec, + project_schema: &SchemaRef, +) -> Result> { + // The src_statistics columns does not match the src schema. + if src_statistics.borrow().column_statistics.len() != src_schema.fields().len() { + return Err(internal_datafusion_err!( + "statistics do not have the same number of columns as the schema, the schema has {:?} fields whereas the statistics have {:?} columns", + src_schema.fields().len(), + src_statistics.borrow().column_statistics.len() + )); + } + + // Check if projection schema matches the len of the subset_selected. + // Do this before using the subset_selected for projecting onto the src_schema. + if project_schema.fields().len() != subset_selected.len() { + return Err(internal_datafusion_err!( + "projected schema cannot be a subset of columns, the schema has {:?} fields whereas the projection indices have {:?} columns", + project_schema.fields().len(), + subset_selected.len() + )); + } + + // Select a subset of columns, erroring if the output column does not exist in the input schema. + let column_statistics = src_schema + .project(subset_selected)? + .fields() + .iter() + .zip(subset_selected) + .enumerate() + .map(|(proj_col_idx, (src_field, src_col_idx))| { + let project_field = &project_schema.fields()[proj_col_idx]; + + // aliasing not permitted + if src_field.name() != project_field.name() { + return Err(internal_datafusion_err!( + "field `{}`@{} does not exist in input schema, found fields [{}]", + project_field.name(), + src_col_idx, + pretty_fmt_fields(src_schema) + )); + } + + // confirm idx is in the col_stats + let Some(col_stats) = + src_statistics.borrow().column_statistics.get(*src_col_idx) + else { + return Err(internal_datafusion_err!( + "missing column statistics for {}@{}", + src_field.name(), + src_col_idx + )); + }; + Ok(col_stats.clone()) + }) + .try_collect()?; + + Ok(Arc::new(Statistics { + num_rows: src_statistics.borrow().num_rows, + total_byte_size: src_statistics.borrow().total_byte_size, + column_statistics, + })) +} + +/// For [`ProjectionExec`](datafusion_physical_plan::projection::ProjectionExec), +/// selectively cast schema projection on [`Statistics`]. +/// +/// This differs from other schema projections, as non-[`Column`] expressions may be present +/// or aliasing may be used. +/// In these circumstances, instead of erroring it returns unknown/absent column_statistics. +/// +/// This code is pulled from a private internal DataFusion function: +/// +pub(super) fn proj_exec_stats<'a>( + mut stats: Statistics, + exprs: impl Iterator, String)>, + projexec_schema: &SchemaRef, +) -> Arc { + let mut primitive_row_size = 0; + let mut primitive_row_size_possible = true; + let mut column_statistics = vec![]; + for (expr, _) in exprs { + let col_stats = if let Some(col) = expr.as_any().downcast_ref::() { + stats.column_statistics[col.index()].clone() + } else { + // TODO stats: estimate more statistics from expressions + // (expressions should compute their statistics themselves) + ColumnStatistics::new_unknown() + }; + column_statistics.push(col_stats); + if let Ok(data_type) = expr.data_type(projexec_schema) { + if let Some(value) = data_type.primitive_width() { + primitive_row_size += value; + continue; + } + } + primitive_row_size_possible = false; + } + + if primitive_row_size_possible { + stats.total_byte_size = + Precision::Exact(primitive_row_size).multiply(&stats.num_rows); + } + stats.column_statistics = column_statistics; + Arc::new(stats) +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::datatypes::{Field, Schema}; + use datafusion_physical_plan::expressions::{col, NoOp}; + + fn build_schema(col_names: Vec<&str>) -> SchemaRef { + let fields = col_names + .into_iter() + .map(|col| Field::new(col, arrow::datatypes::DataType::Int64, false)) + .collect_vec() + .into(); + Arc::new(Schema { + fields, + metadata: Default::default(), + }) + } + + fn build_col_stats( + values: Vec<(&str, i64, i64, usize)>, + ) -> (Arc, SchemaRef) { + let (column_statistics, fields) = values.into_iter().fold( + (vec![], vec![]), + |(mut col_stats, mut fields), (col_name, min, max, nulls)| { + let col_stat = if col_name == "I am an alias" + || col_name == "I am an alias for possible idx bounds-failure" + { + ColumnStatistics::new_unknown() + } else { + ColumnStatistics { + min_value: Precision::Exact(ScalarValue::Int64(Some(min))), + max_value: Precision::Exact(ScalarValue::Int64(Some(max))), + null_count: Precision::Exact(nulls), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + } + }; + + col_stats.push(col_stat); + fields.push(col_name); + (col_stats, fields) + }, + ); + + ( + Arc::new(Statistics { + column_statistics, + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, + }), + build_schema(fields), + ) + } + + #[test] + fn test_retain_all_columns() { + let (src_stats, src_schema) = build_col_stats(vec![ + ("col_a", 0, 24, 1), + ("col_b", 354, 242650, 2), + ("col_c", 67, 1111, 3), + ]); + let (expected_stats, project_schema) = build_col_stats(vec![ + ("col_a", 0, 24, 1), + ("col_b", 354, 242650, 2), + ("col_c", 67, 1111, 3), + ]); + + /* Test: works for subset projection */ + let actual = project_select_subset_of_column_statistics( + &src_stats, + &src_schema, + &vec![0, 1, 2], + &project_schema, + ) + .expect("ok"); + assert_eq!( + actual, expected_stats, + "should be able to project all columns" + ); + + /* Test: works for datasource projection */ + let actual = project_schema_onto_datasrc_statistics( + &src_stats, + &src_schema, + &project_schema, + ) + .expect("ok"); + assert_eq!( + actual, expected_stats, + "should be able to project all columns" + ); + + /* Test: works for projection_exec */ + let exprs = [ + (col("col_a", &src_schema).unwrap(), "col_a".into()), + (col("col_b", &src_schema).unwrap(), "col_b".into()), + (col("col_c", &src_schema).unwrap(), "col_c".into()), + ]; + let actual = proj_exec_stats( + Arc::unwrap_or_clone(src_stats), + exprs.iter(), + &project_schema, + ); + assert_eq!( + actual, expected_stats, + "should be able to project all columns" + ); + } + + #[test] + fn test_remove_and_reorder_columns() { + let (src_stats, src_schema) = build_col_stats(vec![ + ("col_a", 0, 24, 1), + ("col_b", 354, 242650, 2), + ("col_c", 67, 1111, 3), + ]); + let (expected_stats, project_schema) = build_col_stats(vec![ + ("col_c", 67, 1111, 3), + // projection removed col_b + ("col_a", 0, 24, 1), + ]); + + /* Test: works for subset projection */ + let actual = project_select_subset_of_column_statistics( + &src_stats, + &src_schema, + &vec![2, 0], + &project_schema, + ) + .expect("ok"); + assert_eq!( + actual, expected_stats, + "should be able to remove and re-order columns" + ); + + /* Test: correctly errors for subset projection, if the subset_selected doesn't match the projection schema */ + let err = project_select_subset_of_column_statistics( + &src_stats, + &src_schema, + &vec![0, 2], // this is incorrect + &project_schema, + ) + .unwrap_err(); + assert!(err.message().contains( + "field `col_c`@0 does not exist in input schema, found fields [col_a,col_b,col_c]" + )); + + /* Test: works for datasource projection */ + let actual = project_schema_onto_datasrc_statistics( + &src_stats, + &src_schema, + &project_schema, + ) + .expect("ok"); + assert_eq!( + actual, expected_stats, + "should be able to remove and re-order columns" + ); + + /* Test: works for projection_exec */ + let exprs = [ + (col("col_c", &src_schema).unwrap(), "col_c".into()), + (col("col_a", &src_schema).unwrap(), "col_a".into()), + ]; + let actual = proj_exec_stats( + Arc::unwrap_or_clone(src_stats), + exprs.iter(), + &project_schema, + ); + assert_eq!( + actual, expected_stats, + "should be able to remove and re-order columns" + ); + } + + #[test] + fn test_projection_with_aliases() { + /* Test: can handle schema with aliases */ + let (src_stats, src_schema) = + build_col_stats(vec![("col_a", 0, 24, 1), ("col_b", 354, 242650, 2)]); + let (expected_stats, project_schema) = build_col_stats(vec![ + ("col_a", 0, 24, 1), + ("I am an alias", 0, 0, 0), + ("I am an alias for possible idx bounds-failure", 0, 0, 0), + ]); + + /* Test: errors for subset projection, when number of subset_selected don't match the number of projected fields */ + let err = project_select_subset_of_column_statistics( + &src_stats, + &src_schema, + &vec![0], + &project_schema, + ) + .unwrap_err(); + assert!(err.message().contains( + "projected schema cannot be a subset of columns, the schema has 3 fields whereas the projection indices have 1 columns" + )); + + /* Test: errors for subset projection, when field names in the src vs projected schema do not match */ + let err = project_select_subset_of_column_statistics( + &src_stats, + &src_schema, + &vec![0, 1, 1], + &project_schema, + ) + .unwrap_err(); + assert!(err.message().contains( + "field `I am an alias`@1 does not exist in input schema, found fields [col_a,col_b]." + )); + + /* Test: works for datasource projection */ + // ** SPECIAL FOR DATASRC = fills in nulls for aliases missing in filegroup schema ** + let actual = project_schema_onto_datasrc_statistics( + &src_stats, + &src_schema, + &project_schema, + ) + .expect("ok"); + let datasrc_null_columns = ColumnStatistics { + null_count: expected_stats.num_rows, + max_value: Precision::Exact(ScalarValue::Null), + min_value: Precision::Exact(ScalarValue::Null), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + }; + let expected_stats_with_nulls_at_datasrc = Arc::new(Statistics { + num_rows: expected_stats.num_rows, + total_byte_size: expected_stats.total_byte_size, + column_statistics: vec![ + expected_stats.column_statistics[0].clone(), + datasrc_null_columns.clone(), + datasrc_null_columns.clone(), + ], + }); + assert_eq!( + actual, expected_stats_with_nulls_at_datasrc, + "should be able to handle schema with aliases" + ); + assert_eq!( + actual.column_statistics[1], datasrc_null_columns, + "aliased column should have NULL min/max" + ); + + /* Test: works for projection_exec */ + let exprs = [ + (col("col_a", &src_schema).unwrap(), "col_a".into()), + (Arc::new(NoOp::new()), "I am an alias".into()), + ( + Arc::new(NoOp::new()), + "I am an alias for possible idx bounds-failure".into(), + ), + ]; + let actual = proj_exec_stats( + Arc::unwrap_or_clone(src_stats), + exprs.iter(), + &project_schema, + ); + assert_eq!( + actual, expected_stats, + "should be able to handle schema with aliases" + ); + assert_eq!( + actual.column_statistics[1], + ColumnStatistics::new_unknown(), + "aliased column should have unknown statistics" + ); + } + + #[test] + fn test_src_statistics_mismatches_src_schema() { + /* Test: will error if src-statistics and src-schema have a different number of columns, for a non-scan node */ + let (src_stats_2_cols, _) = build_col_stats(vec![ + ("col_a", 0, 24, 1), + ("make an extra column stats", 0, 24, 1), + ]); + let (_, src_schema_1_col) = build_col_stats(vec![("col_a", 0, 24, 1)]); + let (_expected_stats, project_schema) = + build_col_stats(vec![("col_a", 0, 24, 1)]); + let expected_error = + "statistics do not have the same number of columns as the schema"; + let actual_err = project_select_subset_of_column_statistics( + &src_stats_2_cols, + &src_schema_1_col, + &vec![0, 1], + &project_schema, + ) + .unwrap_err(); + assert!(actual_err.message().contains(expected_error)); + } + + /// col_a (0), col_a (1) --> col_a (0), col_a (1) + #[test] + fn test_projection_with_2_fields_having_same_name_and_same_projection_order() { + let (src_stats, src_schema) = build_col_stats(vec![ + ("col_a", 0, 24, 1), // first field with smaller stats + ("col_a", 10, 100, 42), // second field with larger stats + ]); + let (expected_stats, project_schema) = + build_col_stats(vec![("col_a", 0, 24, 1), ("col_a", 10, 100, 42)]); + + /* Test: works for subset projection */ + let actual = project_select_subset_of_column_statistics( + &src_stats, + &src_schema, + &vec![0, 1], + &project_schema, + ) + .expect("ok"); + assert_eq!( + actual, expected_stats, + "should be able to handle schema with same-named fields" + ); + assert_eq!( + actual.column_statistics[0].null_count.get_value(), + Some(&1), + "smaller stats col_a should come first" + ); + assert_eq!( + actual.column_statistics[1].null_count.get_value(), + Some(&42), + "larger stats col_a should come second" + ); + + /* Test: DOES NOT WORK for datasource projection, we take the first matching field_name in the schema */ + let actual = project_schema_onto_datasrc_statistics( + &src_stats, + &src_schema, + &project_schema, + ) + .expect("ok"); + assert_eq!( + actual.column_statistics[0].null_count.get_value(), + Some(&1), + "smaller stats col_a should come first" + ); + assert_eq!( + actual.column_statistics[1].null_count.get_value(), + Some(&1), + "** should take the first matching, which is again the smaller stats **" + ); + + /* Test: works for projection_exec */ + let exprs = [ + (Arc::new(Column::new("col_a", 0)) as _, "col_a_idx0".into()), + (Arc::new(Column::new("col_a", 1)) as _, "col_a_idx1".into()), + ]; + let actual = proj_exec_stats( + Arc::unwrap_or_clone(src_stats), + exprs.iter(), + &project_schema, + ); + assert_eq!( + actual, expected_stats, + "should be able to handle schema with same-named fields" + ); + assert_eq!( + actual.column_statistics[0].null_count.get_value(), + Some(&1), + "smaller stats col_a should come first" + ); + assert_eq!( + actual.column_statistics[1].null_count.get_value(), + Some(&42), + "larger stats col_a should come second" + ); + } + + /// col_a (0), col_a (1) --> col_a (1), col_a (0) + #[test] + fn test_projection_with_2_fields_having_same_name_and_inverse_projection_order() { + let (src_stats, src_schema) = build_col_stats(vec![ + ("col_a", 0, 24, 1), // first field with smaller stats + ("col_a", 10, 100, 42), // second field with larger stats + ]); + // project as [1,0] a.k.a. reverse the columns + let (expected_stats, project_schema) = + build_col_stats(vec![("col_a", 10, 100, 42), ("col_a", 0, 24, 1)]); + + /* Test: works for subset projection */ + let actual = project_select_subset_of_column_statistics( + &src_stats, + &src_schema, + &vec![1, 0], // reverse the columns + &project_schema, + ) + .expect("ok"); + assert_eq!( + actual, expected_stats, + "should be able to handle schema with same-named fields, reversed ordering" + ); + assert_eq!( + actual.column_statistics[0].null_count.get_value(), + Some(&42), + "bigger stats col_a should come first" + ); + + /* Test: DOES NOT WORK for datasource projection, we take the first matching field_name in the schema */ + let actual = project_schema_onto_datasrc_statistics( + &src_stats, + &src_schema, + &project_schema, + ) + .expect("ok"); + assert_eq!( + actual.column_statistics[0].null_count.get_value(), + Some(&1), + "smaller stats col_a should come first" + ); + assert_eq!( + actual.column_statistics[1].null_count.get_value(), + Some(&1), + "** should take the first matching, which is again the smaller stats **" + ); + + /* Test: NOT POSSIBLE for chunk order projection, since it's a pass-thru with no re-ordering of columns */ + + /* Test: works for projection_exec */ + let exprs = [ + (Arc::new(Column::new("col_a", 1)) as _, "col_a_idx1".into()), + (Arc::new(Column::new("col_a", 0)) as _, "col_a_idx0".into()), + ]; + let actual = proj_exec_stats( + Arc::unwrap_or_clone(src_stats), + exprs.iter(), + &project_schema, + ); + assert_eq!( + actual, expected_stats, + "should be able to handle schema with same-named fields, reversed ordering" + ); + assert_eq!( + actual.column_statistics[0].null_count.get_value(), + Some(&42), + "bigger stats col_a should come first" + ); + } +} diff --git a/datafusion/physical-optimizer/src/progressive_evaluation/statistics/util.rs b/datafusion/physical-optimizer/src/progressive_evaluation/statistics/util.rs new file mode 100644 index 000000000000..e382c2428e5f --- /dev/null +++ b/datafusion/physical-optimizer/src/progressive_evaluation/statistics/util.rs @@ -0,0 +1,111 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::{borrow::Borrow, fmt::Debug, sync::Arc}; + +use arrow::datatypes::SchemaRef; +use datafusion_common::{stats::Precision, ColumnStatistics, Statistics}; +use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; +use itertools::Itertools; + +pub(super) fn partition_count(plan: &dyn ExecutionPlan) -> usize { + plan.output_partitioning().partition_count() +} + +pub(super) fn pretty_fmt_fields(schema: &SchemaRef) -> String { + schema.fields().iter().map(|field| field.name()).join(",") +} + +pub(super) fn make_column_statistics_inexact( + column_statistics: Vec, +) -> Vec { + column_statistics + .into_iter() + .map( + |ColumnStatistics { + null_count, + min_value, + max_value, + sum_value, + distinct_count, + }| { + ColumnStatistics { + null_count: make_inexact_or_keep_absent(null_count), + min_value: make_inexact_or_keep_absent(min_value), + max_value: make_inexact_or_keep_absent(max_value), + sum_value: make_inexact_or_keep_absent(sum_value), + distinct_count: make_inexact_or_keep_absent(distinct_count), + } + }, + ) + .collect_vec() +} + +fn make_inexact_or_keep_absent( + val: Precision, +) -> Precision { + match val { + Precision::Exact(val) => Precision::Inexact(val), + _ => val, + } +} + +/// Merge a collection of [`Statistics`] into a single stat. +/// +/// This takes statistics references, which may or may not be arc'ed. +pub(super) fn merge_stats_collection< + T: Borrow + Clone + Into>, +>( + mut stats: impl Iterator, + target_schema: &SchemaRef, +) -> Statistics { + let Some(start) = stats.next() else { + // stats is empty + return Statistics::new_unknown(target_schema); + }; + stats.fold(start.borrow().clone(), |a, b| merge_stats(a, b.borrow())) +} + +/// Merge together two [`Statistics`]. +fn merge_stats(a: Statistics, b: &Statistics) -> Statistics { + assert_eq!( + a.column_statistics.len(), + b.column_statistics.len(), + "failed to merge statistics, due to different column schema" + ); + + Statistics { + num_rows: a.num_rows.add(&b.num_rows), + total_byte_size: a.total_byte_size.add(&b.total_byte_size), + column_statistics: a + .column_statistics + .into_iter() + .zip(b.column_statistics.iter()) + .map(|(a, b)| merge_col_stats(a, b)) + .collect_vec(), + } +} + +fn merge_col_stats(a: ColumnStatistics, b: &ColumnStatistics) -> ColumnStatistics { + ColumnStatistics { + null_count: a.null_count.add(&b.null_count), + min_value: a.min_value.min(&b.min_value), + max_value: a.max_value.max(&b.max_value), + sum_value: a.sum_value.add(&b.sum_value), + distinct_count: Precision::Absent, + } +} diff --git a/datafusion/physical-optimizer/src/progressive_evaluation/util.rs b/datafusion/physical-optimizer/src/progressive_evaluation/util.rs new file mode 100644 index 000000000000..0f53dc29b89d --- /dev/null +++ b/datafusion/physical-optimizer/src/progressive_evaluation/util.rs @@ -0,0 +1,428 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use datafusion_common::{ + error::Result, + tree_node::{Transformed, TreeNodeRecursion}, +}; +use datafusion_datasource::{ + file_groups::FileGroup, file_scan_config::FileScanConfig, source::DataSourceExec, + PartitionedFile, +}; +use datafusion_physical_expr::LexOrdering; +use datafusion_physical_plan::{sorts::sort::SortExec, ExecutionPlan}; + +use itertools::Itertools; + +use crate::progressive_evaluation::extract_ranges::extract_ranges_from_files; + +// This is an optimization for the most recent value query `ORDER BY time DESC/ASC LIMIT n` +// See: https://github.com/influxdata/influxdb_iox/issues/12205 +// The observation is recent data is mostly in first file, so the plan should avoid reading the others unless necessary +// +/// This function is to split all files in the same ParquetExec into different groups/DF partitions and +/// set the `preserve_partitioning` so they will be executed sequentially. The files will later be re-ordered +/// (if non-overlapping) by lexical range. +pub fn split_parquet_files( + plan: Arc, + ordering_req: &LexOrdering, +) -> Result>> { + if let Some(sort_exec) = plan.as_any().downcast_ref::() { + if !sort_exec + .properties() + .equivalence_properties() + .ordering_satisfy(ordering_req) + { + // halt on DAG branch + Ok(Transformed::new(plan, false, TreeNodeRecursion::Stop)) + } else { + // continue down + Ok(Transformed::no(plan)) + } + } else if let Some(parquet_exec) = plan.as_any().downcast_ref::() { + if let Some(transformed) = + transform_parquet_exec_single_file_each_group(parquet_exec, ordering_req)? + { + Ok(Transformed::yes(transformed)) + } else { + Ok(Transformed::no(plan)) + } + } else { + Ok(Transformed::no(plan)) + } +} + +/// Transform a ParquetExec with N files in various groupings, +/// into a ParquetExec into N groups each include one file. +/// +/// The function is only called when the plan does not include DeduplicateExec and includes only one ParquetExec. +/// +/// This function will return error if +/// - There are no statsitics for the given column (including the when the column is missing from the file +/// and produce null values that leads to absent statistics) +/// - Some files overlap (the min/max time ranges are disjoint) +/// - There is a DeduplicateExec in the plan which means the data of the plan overlaps +/// +/// The output ParquetExec's are ordered such that the file with the most recent time ranges is read first +/// +/// For example +/// ```text +/// ParquetExec(groups=[[file1,file2], [file3]]) +/// ``` +/// Is rewritten so each file is in its own group and the files are ordered by time range +/// ```text +/// ParquetExec(groups=[[file1], [file2], [file3]]) +/// ``` +fn transform_parquet_exec_single_file_each_group( + datasrc_exec: &DataSourceExec, + ordering_req: &LexOrdering, +) -> Result>> { + if datasrc_exec + .properties() + .output_partitioning() + .partition_count() + == 1 + { + return Ok(None); + } + + // Extract partitioned files from the ParquetExec + let Some(base_config) = datasrc_exec + .data_source() + .as_any() + .downcast_ref::() + else { + return Ok(None); + }; + let files = base_config + .file_groups + .iter() + .flat_map(|group| group.files()) + .collect_vec(); + let schema = Arc::clone(&base_config.file_schema); + + // extract disjoint lexical ranges + // if cannot find, then is not disjoint + let Some(lexical_ranges) = extract_ranges_from_files(ordering_req, &files, schema)? + else { + return Ok(None); + }; + + // reorder partitioned files by lexical indices + let indices = lexical_ranges.indices(); + assert_eq!( + indices.len(), + files.len(), + "should have every file listed in the sorted indices" + ); + let mut new_partitioned_file_groups = files + .into_iter() + .enumerate() + .map(|(file_idx, file)| { + ( + indices + .iter() + .position(|sorted_idx| *sorted_idx == file_idx) + .expect("file should be listed in indices"), + file, + ) + }) + .collect::>(); + new_partitioned_file_groups.sort_by_key(|(idx, _)| *idx); + + // create new file grouping + let new_partitioned_file_groups = new_partitioned_file_groups + .into_iter() + .map(|(_, file)| { + // each file group has 1 file + build_file_group_with_stats(file) + }) + .collect_vec(); + + // Assigned new partitioned file groups to the new base config + let mut new_base_config = base_config.clone(); + new_base_config.file_groups = new_partitioned_file_groups; + + Ok(Some(DataSourceExec::from_data_source(new_base_config))) +} + +fn build_file_group_with_stats(file: &PartitionedFile) -> FileGroup { + let mut group = FileGroup::new(vec![file.clone()]); + if let Some(stats) = &file.statistics { + group = group.with_statistics(Arc::clone(stats)) + } + group +} + +#[cfg(test)] +pub(crate) mod test_utils { + use arrow::datatypes::DataType; + use arrow::datatypes::Field; + use arrow::datatypes::SchemaRef; + use datafusion_common::ScalarValue; + use datafusion_common::{stats::Precision, Statistics}; + use datafusion_datasource::file_scan_config::FileScanConfig; + use datafusion_datasource::file_scan_config::FileScanConfigBuilder; + use datafusion_datasource::source::DataSourceExec; + use datafusion_datasource::PartitionedFile; + use datafusion_datasource_parquet::source::ParquetSource; + use datafusion_execution::object_store::ObjectStoreUrl; + use datafusion_physical_expr::LexOrdering; + use datafusion_physical_plan::{ + coalesce_batches::CoalesceBatchesExec, + filter::FilterExec, + joins::CrossJoinExec, + limit::LocalLimitExec, + projection::ProjectionExec, + repartition::RepartitionExec, + sorts::{sort::SortExec, sort_preserving_merge::SortPreservingMergeExec}, + union::UnionExec, + Partitioning, PhysicalExpr, + }; + use datafusion_physical_plan::{ColumnStatistics, ExecutionPlan}; + + use itertools::Itertools; + + use std::{ + fmt::{self, Display, Formatter}, + sync::Arc, + }; + + /// Return a schema with a single column `a` of type int64. + pub fn single_column_schema() -> SchemaRef { + Arc::new(arrow::datatypes::Schema::new(vec![Field::new( + "a", + DataType::Int64, + true, + )])) + } + + #[derive(Debug, Copy, Clone)] + pub struct SortKeyRange { + pub min: Option, + pub max: Option, + pub null_count: usize, + } + + impl From for ColumnStatistics { + fn from(val: SortKeyRange) -> Self { + Self { + null_count: Precision::Exact(val.null_count), + max_value: Precision::Exact(ScalarValue::Int32(val.max)), + min_value: Precision::Exact(ScalarValue::Int32(val.min)), + sum_value: Precision::Absent, + distinct_count: Precision::Absent, + } + } + } + + impl Display for SortKeyRange { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!(f, "({:?})->({:?})", self.min, self.max)?; + if self.null_count > 0 { + write!(f, " null_count={}", self.null_count)?; + } + Ok(()) + } + } + + /// Create a single parquet, with a given ordering and using the statistics from the [`SortKeyRange`] + pub fn parquet_exec_with_sort_with_statistics( + output_ordering: Vec, + key_ranges: &[&SortKeyRange], + ) -> Arc { + parquet_exec_with_sort_with_statistics_and_schema( + &single_column_schema(), + output_ordering, + key_ranges, + ) + } + + pub type RangeForMultipleColumns = Vec; // vec![col0, col1, col2] + pub struct PartitionedFilesAndRanges { + pub per_file: Vec, + } + + /// Create a single parquet exec, with multiple parquet, with a given ordering and using the statistics from the [`SortKeyRange`]. + /// Assumes a single column schema. + pub fn parquet_exec_with_sort_with_statistics_and_schema( + schema: &SchemaRef, + output_ordering: Vec, + key_ranges_for_single_column_multiple_files: &[&SortKeyRange], // VecPerFile + ) -> Arc { + let per_file_ranges = PartitionedFilesAndRanges { + per_file: key_ranges_for_single_column_multiple_files + .iter() + .map(|single_col_range_per_file| vec![**single_col_range_per_file]) + .collect_vec(), + }; + + let file_scan_config = file_scan_config(schema, output_ordering, per_file_ranges); + + DataSourceExec::from_data_source(file_scan_config) + } + + /// Create a file scan config with a given file [`SchemaRef`], ordering, + /// and [`ColumnStatistics`] for multiple columns. + pub fn file_scan_config( + schema: &SchemaRef, + output_ordering: Vec, + multiple_column_key_ranges_per_file: PartitionedFilesAndRanges, + ) -> FileScanConfig { + let PartitionedFilesAndRanges { per_file } = multiple_column_key_ranges_per_file; + let mut statistics = Statistics::new_unknown(schema); + let mut file_groups = Vec::with_capacity(per_file.len()); + + // cummulative statistics for the entire parquet exec, per sort key + let num_sort_keys = per_file[0].len(); + let mut cum_null_count = vec![0; num_sort_keys]; + let mut cum_min = vec![None; num_sort_keys]; + let mut cum_max = vec![None; num_sort_keys]; + + // iterate thru files, creating the PartitionedFile and the associated statistics + for (file_idx, multiple_column_key_ranges_per_file) in + per_file.into_iter().enumerate() + { + // gather stats for all columns + let mut per_file_col_stats = Vec::with_capacity(num_sort_keys); + for (col_idx, key_range) in + multiple_column_key_ranges_per_file.into_iter().enumerate() + { + let SortKeyRange { + min, + max, + null_count, + } = key_range; + + // update per file stats + per_file_col_stats.push(ColumnStatistics { + null_count: Precision::Exact(null_count), + min_value: Precision::Exact(ScalarValue::Int32(min)), + max_value: Precision::Exact(ScalarValue::Int32(max)), + ..Default::default() + }); + + // update cummulative statistics for entire parquet exec + cum_min[col_idx] = match (cum_min[col_idx], min) { + (None, x) => x, + (x, None) => x, + (Some(a), Some(b)) => Some(std::cmp::min(a, b)), + }; + cum_max[col_idx] = match (cum_max[col_idx], max) { + (None, x) => x, + (x, None) => x, + (Some(a), Some(b)) => Some(std::cmp::max(a, b)), + }; + cum_null_count[col_idx] += null_count; + } + + // Create single file with statistics. + let mut file = PartitionedFile::new(format!("{file_idx}.parquet"), 100); + file.statistics = Some(Arc::new(Statistics { + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, + column_statistics: per_file_col_stats, + })); + file_groups.push(vec![file].into()); + } + + // add stats, for the whole parquet exec, for all columns + for col_idx in 0..num_sort_keys { + statistics.column_statistics[col_idx] = ColumnStatistics { + null_count: Precision::Exact(cum_null_count[col_idx]), + min_value: Precision::Exact(ScalarValue::Int32(cum_min[col_idx])), + max_value: Precision::Exact(ScalarValue::Int32(cum_max[col_idx])), + ..Default::default() + }; + } + + FileScanConfigBuilder::new( + ObjectStoreUrl::parse("test:///").unwrap(), + Arc::clone(schema), + Arc::new(ParquetSource::new(Default::default())), + ) + .with_file_groups(file_groups) + .with_output_ordering(output_ordering) + .with_statistics(statistics) + .build() + } + + pub fn union_exec(input: Vec>) -> Arc { + Arc::new(UnionExec::new(input)) + } + + pub fn sort_exec( + sort_exprs: &LexOrdering, + input: &Arc, + preserve_partitioning: bool, + ) -> Arc { + let new_sort = SortExec::new(sort_exprs.clone(), Arc::clone(input)) + .with_preserve_partitioning(preserve_partitioning); + Arc::new(new_sort) + } + + pub fn coalesce_exec(input: &Arc) -> Arc { + Arc::new(CoalesceBatchesExec::new(Arc::clone(input), 10)) + } + + pub fn filter_exec( + input: &Arc, + predicate: Arc, + ) -> Arc { + Arc::new(FilterExec::try_new(predicate, Arc::clone(input)).unwrap()) + } + + pub fn limit_exec( + input: &Arc, + fetch: usize, + ) -> Arc { + Arc::new(LocalLimitExec::new(Arc::clone(input), fetch)) + } + + pub fn proj_exec( + input: &Arc, + projects: Vec<(Arc, String)>, + ) -> Arc { + Arc::new(ProjectionExec::try_new(projects, Arc::clone(input)).unwrap()) + } + + pub fn spm_exec( + input: &Arc, + sort_exprs: &LexOrdering, + ) -> Arc { + Arc::new(SortPreservingMergeExec::new( + sort_exprs.clone(), + Arc::clone(input), + )) + } + + pub fn repartition_exec( + input: &Arc, + partitioning: Partitioning, + ) -> Arc { + Arc::new(RepartitionExec::try_new(Arc::clone(input), partitioning).unwrap()) + } + + pub fn crossjoin_exec( + left: &Arc, + right: &Arc, + ) -> Arc { + Arc::new(CrossJoinExec::new(Arc::clone(left), Arc::clone(right))) + } +} diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index b99bef38f324..acff6234a7e2 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -591,7 +591,7 @@ impl EmbeddedProjection for FilterExec { /// converted to closed bounds. If a lower/upper bound is initially open, it /// is adjusted by using the next/previous value for its data type to convert /// it into a closed bound. -fn collect_new_statistics( +pub fn collect_new_statistics( input_column_stats: &[ColumnStatistics], analysis_boundaries: Vec, ) -> Vec {